[ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

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

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

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

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

Change-Id: I73e128029a46f45e6b68c23dfb9310d5de10582f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2393
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 3c981d4..1010a84 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -24,6 +24,7 @@
 import java.util.List;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
 import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
@@ -31,6 +32,7 @@
 import org.apache.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import org.apache.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import org.apache.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
+import org.apache.asterix.optimizer.rules.CheckFullParallelSortRule;
 import org.apache.asterix.optimizer.rules.CheckInsertUpsertReturningRule;
 import org.apache.asterix.optimizer.rules.ConstantFoldingRule;
 import org.apache.asterix.optimizer.rules.CountVarToCountOneRule;
@@ -338,7 +340,9 @@
         physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
-        physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
+        physicalRewritesAllLevels.add(new CheckFullParallelSortRule());
+        physicalRewritesAllLevels
+                .add(new EnforceStructuralPropertiesRule(BuiltinFunctions.RANGE_MAP, BuiltinFunctions.LOCAL_SAMPLING));
         physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
         physicalRewritesAllLevels.add(new RemoveUnnecessarySortMergeExchange());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
@@ -377,6 +381,7 @@
         prepareForJobGenRewrites.add(new SetAlgebricksPhysicalOperatorsRule());
         prepareForJobGenRewrites
                 .add(new IsolateHyracksOperatorsRule(HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
+        prepareForJobGenRewrites.add(new FixReplicateOperatorOutputsRule());
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
         // Re-infer all types, so that, e.g., the effect of not-is-null is
         // propagated.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java
new file mode 100644
index 0000000..7b13a2b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * <pre>
+ * Description:
+ *      This rule checks whether full parallel sort is applicable to {@link OrderOperator}. It disables full parallel
+ *      sort when a limit operator or running aggregate operator is present in the parents of the order operator.
+ * Pre-conditions:
+ *      None.
+ * Post-requirements:
+ *      1. {@link org.apache.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule}
+ * </pre>
+ */
+public class CheckFullParallelSortRule implements IAlgebraicRewriteRule {
+    private final List<AbstractLogicalOperator> parents = new ArrayList<>();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        parents.add((AbstractLogicalOperator) opRef.getValue());
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx) throws AlgebricksException {
+        parents.remove(parents.size() - 1);
+        AbstractLogicalOperator orderOp = (AbstractLogicalOperator) opRef.getValue();
+        if (orderOp.getOperatorTag() == LogicalOperatorTag.ORDER
+                && !orderOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+            // disable when sort output is consumed by limit & running agg op (result should be unpartitioned for limit)
+            AbstractLogicalOperator parent;
+            for (int i = parents.size() - 1; i >= 0; i--) {
+                parent = parents.get(i);
+                if (parent.getOperatorTag() == LogicalOperatorTag.LIMIT
+                        && ((LimitOperator) parent).isTopmostLimitOp()) {
+                    orderOp.getAnnotations().put(OperatorAnnotations.USE_DYNAMIC_RANGE, Boolean.FALSE);
+                    return true;
+                }
+                if (parent.getOperatorTag() == LogicalOperatorTag.RUNNINGAGGREGATE) {
+                    orderOp.getAnnotations().put(OperatorAnnotations.USE_DYNAMIC_RANGE, Boolean.FALSE);
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
index bd79a38..dc135c9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
@@ -140,7 +140,7 @@
             boolean parentFixed = false;
             for (int oldParentIndex = 0; oldParentIndex < replicateOperator.getOutputs().size(); oldParentIndex++) {
                 if (parentsPathToReplicate.contains(replicateOperator.getOutputs().get(oldParentIndex))) {
-                    replicateOperator.getOutputs().get(oldParentIndex).setValue(replicateActualParent.getValue());
+                    replicateOperator.getOutputs().set(oldParentIndex, replicateActualParent);
                     parentFixed = true;
                     updateNumberOfParentsDone(replicateOperator);
                     break;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 8372851..0c91e9b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -34,11 +34,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -101,12 +102,10 @@
     private class IllegalNonfunctionalFunctionSweeperOperatorVisitor implements ILogicalOperatorVisitor<Void, Void> {
 
         private void sweepExpression(ILogicalExpression expr, ILogicalOperator op) throws AlgebricksException {
-            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                if (!expr.isFunctional()) {
-                    AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
-                            "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
-                }
+            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL && !expr.isFunctional()) {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
+                        "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
             }
         }
 
@@ -308,6 +307,12 @@
         public Void visitTokenizeOperator(TokenizeOperator op, Void tag) throws AlgebricksException {
             return null;
         }
+
+        @Override
+        public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+            sweepExpression(op.getRangeMapExpression().getValue(), op);
+            return null;
+        }
     }
 
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 4d646f3..22ef303 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -59,6 +59,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -642,6 +643,12 @@
         return visitSingleInputOperator(op);
     }
 
+    @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                "Forward operator should have been disqualified for this rewriting!");
+    }
+
     /**
      * Wraps an AggregateOperator or RunningAggregateOperator with a group-by
      * operator where the group-by keys are variables in keyVarsToEnforce. Note
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 610fea2..b862a6f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -42,6 +42,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -52,6 +53,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
@@ -63,23 +65,19 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
 
-/*
-    This visitor inlines the input <code>nts</code> in the query plan rooted
-    at the operator being visited, with the query plan rooted at the input
-    <code>subplanInputOperator</code>.
-
-    The visitor ensures that:
-    1. live variables at <code>subplanInputOperator</code> are
-    propagated to the top-most join operator in the query plan rooted
-    at the operator being visited;
-    2. no available tuple at <code>subplanInputOperator</code> get lost along the
-    pipeline to the top-most join operator in the query plan rooted
-    at the operator being visited.
-*/
+/**
+ * This visitor inlines the input {@code nts} in the query plan rooted at the operator being visited,
+ * with the query plan rooted at the input {@code subplanInputOperator}.
+ *
+ * The visitor ensures that:
+ * 1. live variables at {@code subplanInputOperator} are propagated to the top-most join operator in the query plan
+ * rooted at the operator being visited.
+ * 2. no available tuple at {@code subplanInputOperator} get lost along the pipeline to the top-most join operator
+ * in the query plan rooted at the operator being visited.
+ */
 class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisitor<ILogicalOperator, Void> {
     // The optimization context.
     private final IOptimizationContext context;
@@ -380,6 +378,12 @@
         return visitSingleInputOperator(op);
     }
 
+    @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new UnsupportedOperationException(
+                "Nested subplans with a forward operator should have been disqualified for this rewriting!");
+    }
+
     private ILogicalOperator visitSingleInputOperator(ILogicalOperator op) throws AlgebricksException {
         if (op.getInputs().size() == 1) {
             // Deals with single input operators.
@@ -395,8 +399,7 @@
     }
 
     /**
-     * Inject varaibles to indicate non-matches for the right branch of
-     * a left-outer join.
+     * Inject variables to indicate non-matches for the right branch of a left-outer join.
      *
      * @param joinOp
      *            the leftouter join operator.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index 44bfbe4..e2b104d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -18,16 +18,19 @@
  */
 package org.apache.asterix.optimizer.rules.subplan;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -229,6 +232,12 @@
         return visitInputs(op);
     }
 
+    @Override
+    public Boolean visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                "Forward operator should have been disqualified for this rewriting!");
+    }
+
     private boolean visitInputs(ILogicalOperator op) throws AlgebricksException {
         for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
             if (childRef.getValue().accept(this, null)) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 91de474..19fe02d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -38,7 +38,6 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -69,6 +68,7 @@
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSource;
@@ -1163,9 +1163,9 @@
         }
         if (oc.getRangeMap() != null) {
             Iterator<OrderModifier> orderModifIter = oc.getModifierList().iterator();
-            boolean ascending = (orderModifIter.next() == OrderModifier.ASC);
+            boolean ascending = orderModifIter.next() == OrderModifier.ASC;
             RangeMapBuilder.verifyRangeOrder(oc.getRangeMap(), ascending);
-            ord.getAnnotations().put(OperatorAnnotations.USE_RANGE_CONNECTOR, oc.getRangeMap());
+            ord.getAnnotations().put(OperatorAnnotations.USE_STATIC_RANGE, oc.getRangeMap());
         }
         return new Pair<>(ord, null);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index cfec2de..9727d46 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -36,7 +36,6 @@
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.config.CompilerProperties;
-import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -136,7 +135,8 @@
     private static final Set<String> CONFIGURABLE_PARAMETER_NAMES =
             ImmutableSet.of(CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
                     CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
-                    CompilerProperties.COMPILER_PARALLELISM_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                    CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+                    CompilerProperties.COMPILER_SORT_SAMPLES_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
                     FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
                     StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
                     FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
@@ -336,12 +336,17 @@
         int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
                 (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
                 compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
-        final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.getPhysicalOptimizationConfig();
+        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig);
+        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
+
+        final PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
         physOptConf.setMaxFramesExternalSort(sortFrameLimit);
         physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
         physOptConf.setMaxFramesForJoin(joinFrameLimit);
         physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
+        physOptConf.setSortParallel(fullParallelSort);
+        physOptConf.setSortSamples(sortNumSamples);
 
         return physOptConf;
     }
@@ -495,6 +500,25 @@
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
     }
 
+    private boolean getSortParallel(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
+        if (valueInQuery != null) {
+            return OptionTypes.BOOLEAN.parse(valueInQuery);
+        }
+        return compilerProperties.getSortParallel();
+    }
+
+    private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
+        if (valueInQuery != null) {
+            int parsedNumSamples = OptionTypes.INTEGER.parse(valueInQuery);
+            if (parsedNumSamples > 0) {
+                return parsedNumSamples;
+            }
+        }
+        return compilerProperties.getSortSamples();
+    }
+
     // Validates if the query contains unsupported query parameters.
     private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
             throws AlgebricksException {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
index 1913683..c7db521 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
@@ -86,6 +86,7 @@
             case INDEX_INSERT_DELETE_UPSERT:
             case INSERT_DELETE_UPSERT:
             case INTERSECT:
+            case FORWARD:
                 return getOperatorRequiredMemory(operator, frameSize);
             case LEFT_OUTER_UNNEST_MAP:
             case UNNEST_MAP:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
index 8b32375..0023a7a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
@@ -38,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -67,13 +68,22 @@
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.util.annotations.NotThreadSafe;
 
+/**
+ * Visits the operator first. Then, it visits all its inputs (pre-order traversal). When it visits an operator, it adds
+ * the operator to the current stage. If the operator is a multi-stage operator, it also adds the operator to a queue
+ * to re-visit the operator again to create the other stage.
+ */
 @NotThreadSafe
 public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void> {
 
-    private static final int JOIN_FIRST_INPUT = 1;
-    private static final int JOIN_SECOND_INPUT = 2;
+    private static final int JOIN_NON_BLOCKING_INPUT = 0;
+    private static final int JOIN_BLOCKING_INPUT = 1;
+    private static final int JOIN_NUM_INPUTS = 2;
+    private static final int FORWARD_NON_BLOCKING_INPUT = 0;
+    private static final int FORWARD_BLOCKING_INPUT = 1;
+    private static final int FORWARD_NUM_INPUTS = 2;
     private final Set<ILogicalOperator> visitedOperators = new HashSet<>();
-    private final LinkedList<ILogicalOperator> pendingBlockingOperators = new LinkedList<>();
+    private final LinkedList<ILogicalOperator> pendingMultiStageOperators = new LinkedList<>();
     private final List<PlanStage> stages = new ArrayList<>();
     private PlanStage currentStage;
     private int stageCounter;
@@ -163,7 +173,7 @@
 
     @Override
     public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
-        // Makes sure that the downstream of a replicate operator is only visited once.
+        // make sure that the downstream of a replicate operator is visited only once.
         if (!visitedOperators.contains(op)) {
             visitedOperators.add(op);
             visit(op);
@@ -175,7 +185,7 @@
 
     @Override
     public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
-        // Makes sure that the downstream of a split operator is only visited once.
+        // make sure that the downstream of a split operator is visited only once.
         if (!visitedOperators.contains(op)) {
             visitedOperators.add(op);
             visit(op);
@@ -300,59 +310,82 @@
         return null;
     }
 
+    @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        visit(op);
+        return null;
+    }
+
     public List<PlanStage> getStages() {
         return stages;
     }
 
     private void visit(ILogicalOperator op) throws AlgebricksException {
         addToStage(op);
-        if (!pendingBlockingOperators.isEmpty()) {
-            final ILogicalOperator firstPending = pendingBlockingOperators.pop();
-            visitBlocking(firstPending);
+        if (!pendingMultiStageOperators.isEmpty()) {
+            final ILogicalOperator firstPending = pendingMultiStageOperators.pop();
+            visitMultiStageOp(firstPending);
         }
     }
 
-    private void visitBlocking(ILogicalOperator blockingOp) throws AlgebricksException {
+    private void visitMultiStageOp(ILogicalOperator multiStageOp) throws AlgebricksException {
         final PlanStage blockingOpStage = new PlanStage(++stageCounter);
-        blockingOpStage.getOperators().add(blockingOp);
+        blockingOpStage.getOperators().add(multiStageOp);
         stages.add(blockingOpStage);
         currentStage = blockingOpStage;
-        switch (blockingOp.getOperatorTag()) {
+        switch (multiStageOp.getOperatorTag()) {
             case INNERJOIN:
             case LEFTOUTERJOIN:
-                // visit only the second input
-                ILogicalOperator joinSecondInput = getJoinOperatorInput(blockingOp, JOIN_SECOND_INPUT);
-                joinSecondInput.accept(this, null);
+                // visit only the blocking input creating a new stage
+                ILogicalOperator newStageOperator = getInputAt(multiStageOp, JOIN_BLOCKING_INPUT, JOIN_NUM_INPUTS);
+                newStageOperator.accept(this, null);
                 break;
             case GROUP:
             case ORDER:
-                visitInputs(blockingOp);
+                visitInputs(multiStageOp);
+                break;
+            case FORWARD:
+                // visit only the blocking input creating a new stage
+                ILogicalOperator newStageOp = getInputAt(multiStageOp, FORWARD_BLOCKING_INPUT, FORWARD_NUM_INPUTS);
+                newStageOp.accept(this, null);
                 break;
             default:
-                throw new IllegalStateException("Unrecognized blocking operator: " + blockingOp.getOperatorTag());
+                throw new IllegalStateException("Unrecognized blocking operator: " + multiStageOp.getOperatorTag());
         }
     }
 
+    /**
+     * Adds the op argument to the current stage. If the operator is a multi-stage, it adds the operator to the pending
+     * list and continues on the branch that is non-blocking (i.e., the branch continuing on the same current stage)
+     * @param op to be added to the current stage
+     * @throws AlgebricksException
+     */
     private void addToStage(ILogicalOperator op) throws AlgebricksException {
         currentStage.getOperators().add(op);
         switch (op.getOperatorTag()) {
             case INNERJOIN:
             case LEFTOUTERJOIN:
-                pendingBlockingOperators.add(op);
+                pendingMultiStageOperators.add(op);
                 // continue on the same stage
-                final ILogicalOperator joinFirstInput = getJoinOperatorInput(op, JOIN_FIRST_INPUT);
-                joinFirstInput.accept(this, null);
+                final ILogicalOperator joinNonBlockingInput = getInputAt(op, JOIN_NON_BLOCKING_INPUT, JOIN_NUM_INPUTS);
+                joinNonBlockingInput.accept(this, null);
                 break;
             case GROUP:
                 if (isBlockingGroupBy((GroupByOperator) op)) {
-                    pendingBlockingOperators.add(op);
+                    pendingMultiStageOperators.add(op);
                     return;
                 }
                 // continue on the same stage
                 visitInputs(op);
                 break;
             case ORDER:
-                pendingBlockingOperators.add(op);
+                pendingMultiStageOperators.add(op);
+                break;
+            case FORWARD:
+                pendingMultiStageOperators.add(op);
+                // continue on the same current stage through the branch that is non-blocking
+                ILogicalOperator nonBlockingInput = getInputAt(op, FORWARD_NON_BLOCKING_INPUT, FORWARD_NUM_INPUTS);
+                nonBlockingInput.accept(this, null);
                 break;
             default:
                 visitInputs(op);
@@ -397,15 +430,16 @@
         return false;
     }
 
-    private ILogicalOperator getJoinOperatorInput(ILogicalOperator op, int inputNum) {
-        if (inputNum != JOIN_FIRST_INPUT && inputNum != JOIN_SECOND_INPUT) {
-            throw new IllegalArgumentException("invalid input number for join operator");
-        }
+    private ILogicalOperator getInputAt(ILogicalOperator op, int inputIndex, int numInputs) {
         final List<Mutable<ILogicalOperator>> inputs = op.getInputs();
-        if (inputs.size() != 2) {
-            throw new IllegalStateException("Join must have exactly two inputs. Current inputs: " + inputs.size());
+        int inSize = inputs.size();
+        if (inSize != numInputs) {
+            throw new IllegalStateException("Op must have exactly " + numInputs + " inputs. Current inputs: " + inSize);
         }
-        return op.getInputs().get(inputNum - 1).getValue();
+        if (inputIndex >= inSize) {
+            throw new IllegalArgumentException("invalid input index for operator");
+        }
+        return inputs.get(inputIndex).getValue();
     }
 
     /**
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
index 89fbcb2..368a244 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -304,6 +305,12 @@
         return null;
     }
 
+    @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
     // Calculates the memory usage for exchange operators.
     private void calculateMemoryUsageForExchange(ExchangeOperator op) throws AlgebricksException {
         visitInternal(op, false);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
index 5f76568..d0adcda 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
@@ -23,7 +23,6 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.asterix.external.library.java.base.ByteArrayAccessibleInputStream;
 import org.apache.asterix.external.parser.JSONDataParser;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.IAObject;
@@ -35,6 +34,7 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
 import com.fasterxml.jackson.databind.JsonNode;
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 6971b2b..8877be8 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -53,6 +53,7 @@
 compiler.groupmemory=160KB
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
+compiler.sort.parallel=false
 messaging.frame.size=4096
 messaging.frame.count=512
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
index 385f6a2..66b01a7 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
@@ -23,6 +23,7 @@
 import java.util.List;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -41,7 +42,7 @@
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx) {
         return new ITuplePartitionComputer() {
             private final List<Integer> destinations =
                     new ArrayList<Integer>(TestPartitionComputerFactory.this.destinations);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index debb3f9..6e0413c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -60,6 +60,11 @@
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+/**
+ * Runs the optimization tests. The current configuration runs the tests with parallel sort disabled.
+ * Note: when adding a new test case and it includes sorting, provide another test case and enable parallel sort in the
+ * query by setting the property (append the test case name with "_ps")
+ */
 @RunWith(Parameterized.class)
 public class OptimizerTest {
 
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..4d8c4ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..a5c5fe4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for secondary btree index in index subtree. p_sort enabled.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp
new file mode 100644
index 0000000..520a3f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448). p_sort enabled.
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
+
+set `compiler.sort.parallel` "true";
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp
new file mode 100644
index 0000000..b986780
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *                  : However, we set the "noindexonly" option to true. So, the index-only plan should not be triggered.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set noindexonly 'true';
+set `compiler.sort.parallel` "true";
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
new file mode 100644
index 0000000..e5c4e0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set `compiler.sort.parallel` "true";
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
index 103aae8..9921e58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
@@ -36,6 +36,9 @@
 create  dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
 
 write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
+
+SET `compiler.sort.parallel` "false";
+
 select element token2.rank
 from  TOKENSRANKEDADM as token1,
       (
@@ -44,5 +47,4 @@
     order by tokenRanked.rank
 ) as token2
 where (token1.token = token2.token)
-order by token2.rank
-;
+order by token2.rank;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..0384667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..578ef74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
new file mode 100644
index 0000000..eb17ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
new file mode 100644
index 0000000..412ffdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
new file mode 100644
index 0000000..44f6164
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..cf8b4a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..0b8ec46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+set `simthreshold` `1`;
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (c.interests ~= ['computers','wine','walking'])
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..27157ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
new file mode 100644
index 0000000..3b861b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
new file mode 100644
index 0000000..f5d9888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2c5ac71
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2348dd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.`referred-topics`,t2.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..5d49319
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp
new file mode 100644
index 0000000..d8a4d6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type test.DBLPType as closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+create  dataset CSX(CSXType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT a.id AS aid, b.id AS bid
+FROM DBLP AS a
+LEFT OUTER UNNEST
+(
+  SELECT VALUE b
+  FROM CSX AS b
+  WHERE a.authors = b.authors
+) AS b
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp
new file mode 100644
index 0000000..ff38e33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type test.DBLPType as closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+create  dataset CSX(CSXType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT a.id AS aid, b.id AS bid
+FROM DBLP AS a
+LEFT OUTER JOIN CSX AS b
+ON a.authors = b.authors
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..bc0f28c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..3deb837
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..99a3497
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Mu')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..4f07d69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
new file mode 100644
index 0000000..a11fec7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)[0]
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
new file mode 100644
index 0000000..c9b6d13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)[0]
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
new file mode 100644
index 0000000..db3a364
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 3)
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..05a8936
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 1)
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..b53584e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+set `simthreshold` `1`;
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (c.nested.interests ~= ['computers','wine','walking'])
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..e682634
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index keyword_index  on DBLP (nested.title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
new file mode 100644
index 0000000..5d57ef6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
new file mode 100644
index 0000000..acafb14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.nested.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..bfd2bf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgNgramIx  on TweetMessages (nested.`message-text`) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.nested.`message-text`,t2.nested.`message-text`,7)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..292925a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index topicKeywordIx  on TweetMessages (nested.`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.nested.`referred-topics`,t2.nested.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..245e3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.nested.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..2481d9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.nested.`sender-location`,n) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
new file mode 100644
index 0000000..5b88828
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
new file mode 100644
index 0000000..bb50b07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountAIx  on TweetMessages (nested.countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
new file mode 100644
index 0000000..dbf7c40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
new file mode 100644
index 0000000..df41b65
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountAIx  on TweetMessages (nested.countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..1d22a19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Mu')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..7e98edf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..b6407d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index keyword_index  on DBLP (nested.title:string?) type keyword enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..0a55178
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgNgramIx  on TweetMessages (nested.`message-text`:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.nested.`message-text`,t2.nested.`message-text`,7)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
new file mode 100644
index 0000000..fe6c65d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
new file mode 100644
index 0000000..955f38c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  CSX as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
new file mode 100644
index 0000000..eb6fe2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
new file mode 100644
index 0000000..a924e19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index_DBLP  on DBLP (title:string?) type ngram (3) enforced;
+
+create  index ngram_index_CSX  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..19143d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.nested.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..0471760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.nested.`sender-location`,n) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
new file mode 100644
index 0000000..2514206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
new file mode 100644
index 0000000..abe7d59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountAIx  on TweetMessages (countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
new file mode 100644
index 0000000..1f0c190
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
new file mode 100644
index 0000000..fd29e2e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountAIx  on TweetMessages (countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..dbb430a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..060465d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..9caf6bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title:string?) type keyword enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2d9b15b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgNgramIx  on TweetMessages (`message-text`:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
new file mode 100644
index 0000000..fe6c65d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
new file mode 100644
index 0000000..1cb8237
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
new file mode 100644
index 0000000..eb6fe2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
new file mode 100644
index 0000000..a924e19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index_DBLP  on DBLP (title:string?) type ngram (3) enforced;
+
+create  index ngram_index_CSX  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..3fd898e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..c59dba4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
new file mode 100644
index 0000000..8c3f6dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : No index join because there's no hint and the probe type is unknown. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-05.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
new file mode 100644
index 0000000..7ba14c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : No index join because the probe type is unknown. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-06.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
new file mode 100644
index 0000000..403a75e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : No index join because there's no hint. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-07.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) = t2.c_s
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
new file mode 100644
index 0000000..a82fe5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (string). p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
new file mode 100644
index 0000000..759ce3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (bigint). p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.c_i64) /*+ indexnl */ = t2.c_i64
+order by t1.c_x, t2.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
new file mode 100644
index 0000000..1b585b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree non-enforced index verification test
+ *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+ *  Expected Result : Success
+ *  Date            : 12th Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-04.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_s on TestOpen(c_s:string);
+
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_s = 'world'
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
new file mode 100644
index 0000000..b14b9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-05.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 = 2
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
new file mode 100644
index 0000000..570c37e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-06.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 > 2
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
new file mode 100644
index 0000000..ffb1cf8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-07.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 > 2.0
+order by t.c_x;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
new file mode 100644
index 0000000..6de6121
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
new file mode 100644
index 0000000..3f1a7847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-09.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2.5
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
new file mode 100644
index 0000000..6049e5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *                  : when multiple indexes match
+  *  Expected Result : Success
+  *  Date            : 29 Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-104.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_3 on TestOpen(c_i64:int64);
+create index idx_4 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2 and t.c_i64 < 3
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
new file mode 100644
index 0000000..afb08ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-10.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_d on TestOpen(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_d >= 3.25
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
new file mode 100644
index 0000000..b921fa3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-11.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 499 and t.c_i8 < 99999
+order by t.c_x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
new file mode 100644
index 0000000..254bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse `gby-using-orderby-desc` if exists;
+create  dataverse `gby-using-orderby-desc`;
+
+use `gby-using-orderby-desc`;
+
+
+create type `gby-using-orderby-desc`.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type `gby-using-orderby-desc`.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key cid on group1;
+
+write output to asterix_nc1:"rttest/gby-using-orderby-desc.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'name':name,'age':age}
+from  Customers as c
+group by c.name as name, c.age as age
+order by name desc,age
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
new file mode 100644
index 0000000..7764086
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
new file mode 100644
index 0000000..3814e1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
new file mode 100644
index 0000000..80cec87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
new file mode 100644
index 0000000..de4604e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
similarity index 67%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
index b92aa6c..271565a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
@@ -16,18 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+/*
+* Description  : Test enabling parallel sort by setting the property
+* Expected Res : Success
+*/
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
 
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
+CREATE TYPE CustomerType AS {
+  c_custkey : integer,
+  c_1 : integer
+};
 
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
+CREATE DATASET Customers(CustomerType) PRIMARY KEY c_custkey;
 
-}
\ No newline at end of file
+SET `compiler.sort.parallel` "true";
+
+SELECT VALUE c
+FROM Customers c
+ORDER BY c.c_1;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
similarity index 67%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
index b92aa6c..e79e872 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
@@ -16,18 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+/*
+* Description  : Test disabling parallel sort by setting the property
+* Expected Res : Success
+*/
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
 
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
+CREATE TYPE CustomerType AS {
+  c_custkey : integer,
+  c_1 : integer
+};
 
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
+CREATE DATASET Customers(CustomerType) PRIMARY KEY c_custkey;
 
-}
\ No newline at end of file
+SET `compiler.sort.parallel` "false";
+
+SELECT VALUE c
+FROM Customers c
+ORDER BY c.c_1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
new file mode 100644
index 0000000..3ee5774
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
+        select element i.l.l_quantity
+        from  g as i
+    )),'sum_base_price':tpch.strict_sum((
+        select element i.l.l_extendedprice
+        from  g as i
+    )),'sum_disc_price':tpch.strict_sum((
+        select element (i.l.l_extendedprice * (1 - i.l.l_discount))
+        from  g as i
+    )),'sum_charge':tpch.strict_sum((
+        select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
+        from  g as i
+    )),'ave_qty':tpch.strict_avg((
+        select element i.l.l_quantity
+        from  g as i
+    )),'ave_price':tpch.strict_avg((
+        select element i.l.l_extendedprice
+        from  g as i
+    )),'ave_disc':tpch.strict_avg((
+        select element i.l.l_discount
+        from  g as i
+    )),'count_order':tpch.strict_count(g)}
+from  LineItem as l
+where (l.l_shipdate <= '1998-09-02')
+/* +hash */
+group by l.l_returnflag as l_returnflag,l.l_linestatus as l_linestatus group as g
+order by l_returnflag,l_linestatus
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp
new file mode 100644
index 0000000..28e60df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : This test case is to verify the fix for issue 1806. p_sort enabled.
+ * https://issues.apache.org/jira/browse/ASTERIXDB-1806
+ * Expected Res : Non-index utilization Plan
+ */
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type LineItemType as closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+SET `import-private-functions` "true";
+SET `compiler.sort.parallel` "true";
+
+SELECT  l_returnflag,
+        l_linestatus,
+        sum(l_quantity) AS sum_qty,
+        sum(l_extendedprice) AS sum_base_price,
+        sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+        sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+        avg(l_quantity) AS ave_qty,
+        avg(l_extendedprice) AS ave_price,
+        avg(l_discount) AS ave_disc,
+        count(1) AS count_order
+FROM  LineItem l
+WHERE inject_failure(l.l_shipdate <= '1998-09-02', l.l_orderkey=5988)
+GROUP BY l_returnflag, l_linestatus
+ORDER BY l_returnflag, l_linestatus;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp
similarity index 66%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp
index b92aa6c..722afae 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp
@@ -16,18 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+set `compiler.sort.parallel` "true";
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
-
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file
+SELECT ds.DatasetName as v1
+FROM Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%"
+UNION ALL
+SELECT ds.DatasetName v1, idx.DatasetName v2, idx.IndexName v3
+FROM Metadata.`Index` idx, Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%" and idx.DatasetName LIKE "Orders%"
+ORDER BY v1, v2, v3;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp
new file mode 100644
index 0000000..b542ec7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : This test case is to verify the fix for issue ASTERIXDB-2408. p_sort enabled.
+ * https://issues.apache.org/jira/browse/ASTERIXDB-2408
+ * Expected Res :
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type testType as open {
+  id: int32
+};
+
+create dataset customers(testType) primary key id;
+create dataset orders(testType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT c.name, (
+  SELECT VALUE o.id
+  FROM orders o
+  WHERE o.customer_id = c.id
+) AS orders
+FROM customers c
+ORDER BY array_count(orders);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp
new file mode 100644
index 0000000..3b78d29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp
new file mode 100644
index 0000000..26974a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp
new file mode 100644
index 0000000..dcddc12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp
new file mode 100644
index 0000000..53c79ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue562
+ * https://code.google.com/p/asterixdb/issues/detail?id=562
+ * Expected Res : SUCCESS
+ * Date         : 15th Jan. 2015
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : integer,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+set `compiler.sort.parallel` "true";
+
+declare function q22_customer_tmp() {
+(
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
+    from  Customer as c
+    with  phone_substr as tpch.substring(c.c_phone,0,2)
+    where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
+)
+};
+with  avg as tpch.strict_avg((
+      select element c.c_acctbal
+      from  Customer as c
+      with  phone_substr as tpch.substring(c.c_phone,0,2)
+      where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
+  ))
+select element {'cntrycode':cntrycode,'numcust':tpch.strict_count(g),'totacctbal':tpch.strict_sum((
+        select element i.ct.c_acctbal
+        from  g as i
+    ))}
+from  tpch.q22_customer_tmp() as ct
+where (tpch.strict_count((
+    select element o
+    from  Orders as o
+    where (ct.c_custkey = o.o_custkey)
+)) = 0)
+group by ct.cntrycode as cntrycode group as g
+order by cntrycode
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..2a61fa5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..3e1c881
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
new file mode 100644
index 0000000..c14a25e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
+ * Expected Res : Success
+ * Date         : 21th December 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where  /*+ skip-index */ test.contains(o.title,'Multimedia')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp
new file mode 100644
index 0000000..71bcb4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.EmploymentType as
+{
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type TinySocial.FacebookUserType as
+ closed {
+  id : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+set `compiler.sort.parallel` "true";
+
+with  lonelyusers as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  ),
+      lonelyusers2 as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  )
+select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
+from  lonelyusers as l1,
+      lonelyusers2 as l2
+where (l1.id < l2.id)
+order by l1.id,l2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp
new file mode 100644
index 0000000..7fb60be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: integer,
+  o_custkey: integer,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: integer,
+  o_comment: string
+};
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: integer,
+  c_name: string,
+  c_address: string,
+  c_nationkey: integer,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+};
+
+CREATE EXTERNAL DATASET Customer(CustomerType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET Orders(OrderType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+SET `compiler.sort.parallel` "true";
+
+WITH q22_customer_tmp AS
+(
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
+    FROM  Customer
+)
+,
+avg AS (
+        SELECT ELEMENT AVG(c_acctbal)
+        FROM  Customer
+        WHERE c_acctbal > 0.0
+)[0]
+SELECT  cntrycode, count(ct) AS numcust, SUM(c_acctbal) AS totacctbal
+FROM  q22_customer_tmp AS ct
+WHERE c_acctbal > avg
+      AND EXISTS (SELECT * FROM Orders o WHERE o.o_custkey = ct.c_custkey)
+GROUP BY cntrycode
+ORDER BY cntrycode;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp
new file mode 100644
index 0000000..308a618
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp
new file mode 100644
index 0000000..297116a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp
new file mode 100644
index 0000000..d0d190d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp
new file mode 100644
index 0000000..2f40400
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp
new file mode 100644
index 0000000..6ef533a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp
new file mode 100644
index 0000000..4c4d754
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp
new file mode 100644
index 0000000..8d852f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS {
+  o_orderkey: integer
+};
+
+CREATE TYPE CustomerType AS {
+  c_custkey: integer
+};
+
+CREATE COLLECTION Customers(CustomerType) PRIMARY KEY c_custkey;
+CREATE COLLECTION Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT c.c_custkey customer_name
+FROM Customers c
+WHERE c.c_custkey IN (
+    SELECT VALUE o.o_custkey
+    FROM Orders o
+    WHERE c.c_custkey = o.o_custkey
+  )  AND c.c_nationkey = 5
+ORDER BY customer_name;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp
similarity index 61%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
rename to asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp
index 5c5f34b..876ffbd 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp
@@ -16,20 +16,29 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.common.data.partition.range;
 
-import org.apache.hyracks.data.std.api.IPointable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
 
-public interface IRangeMap {
-    public IPointable getFieldSplit(int columnIndex, int splitIndex);
+CREATE TYPE OrderType AS {
+  o_orderkey: integer
+};
 
-    public int getSplitCount();
+CREATE TYPE CustomerType AS {
+  c_custkey: integer
+};
 
-    public byte[] getByteArray(int columnIndex, int splitIndex);
 
-    public int getStartOffset(int columnIndex, int splitIndex);
+CREATE COLLECTION Customers(CustomerType) PRIMARY KEY c_custkey;
+CREATE COLLECTION Orders(OrderType) PRIMARY KEY o_orderkey;
 
-    public int getLength(int columnIndex, int splitIndex);
+SET `compiler.sort.parallel` "true";
 
-    public int getTag(int columnIndex, int splitIndex);
-}
+SELECT c.c_custkey customer_name
+FROM Customers c
+WHERE c.c_custkey IN (
+    SELECT VALUE o.o_custkey
+    FROM Orders o
+  )  AND c.c_nationkey = 5
+ORDER BY customer_name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp
new file mode 100644
index 0000000..3fcaee3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: integer,
+  o_custkey: integer,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: integer,
+  o_comment: string
+};
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: integer,
+  c_name: string,
+  c_address: string,
+  c_nationkey: integer,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+};
+
+CREATE EXTERNAL DATASET Customer(CustomerType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET Orders(OrderType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+SET `compiler.sort.parallel` "true";
+
+WITH q22_customer_tmp AS
+(
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
+    FROM  Customer
+)
+,
+avg AS (
+        SELECT ELEMENT AVG(c_acctbal)
+        FROM  Customer
+        WHERE c_acctbal > 0.0
+)[0]
+SELECT  cntrycode, count(ct) AS numcust, SUM(c_acctbal) AS totacctbal
+FROM  q22_customer_tmp AS ct
+WHERE c_acctbal > avg
+      AND NOT EXISTS (SELECT * FROM Orders o WHERE o.o_custkey = ct.c_custkey)
+GROUP BY cntrycode
+ORDER BY cntrycode;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp
new file mode 100644
index 0000000..efe7b00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse tpcds if exists;
+create dataverse tpcds;
+use tpcds;
+
+create type tpcds.catalog_sales_type as closed {
+    cs_sold_date_sk:           bigint?,
+    cs_sold_time_sk:           bigint?,
+    cs_ship_date_sk:           bigint?,
+    cs_bill_customer_sk:       bigint?,
+    cs_bill_cdemo_sk:          bigint?,
+    cs_bill_hdemo_sk:          bigint?,
+    cs_bill_addr_sk:           bigint?,
+    cs_ship_customer_sk:       bigint?,
+    cs_ship_cdemo_sk:          bigint?,
+    cs_ship_hdemo_sk:          bigint?,
+    cs_ship_addr_sk:           bigint?,
+    cs_call_center_sk:         bigint?,
+    cs_catalog_page_sk:        bigint?,
+    cs_ship_mode_sk:           bigint?,
+    cs_warehouse_sk:           bigint?,
+    cs_item_sk:                bigint,
+    cs_promo_sk:               bigint?,
+    cs_order_number:           bigint,
+    cs_quantity:               bigint?,
+    cs_wholesale_cost:         double?,
+    cs_list_price:             double?,
+    cs_sales_price:            double?,
+    cs_ext_discount_amt:       double?,
+    cs_ext_sales_price:        double?,
+    cs_ext_wholesale_cost:     double?,
+    cs_ext_list_price:         double?,
+    cs_ext_tax:                double?,
+    cs_coupon_amt:             double?,
+    cs_ext_ship_cost:          double?,
+    cs_net_paid:               double?,
+    cs_net_paid_inc_tax:       double?,
+    cs_net_paid_inc_ship:      double?,
+    cs_net_paid_inc_ship_tax:  double?,
+    cs_net_profit:             double?
+};
+
+create type tpcds.catalog_returns_type as closed {
+    cr_returned_date_sk:       bigint?,
+    cr_returned_time_sk:       bigint?,
+    cr_item_sk:                bigint,
+    cr_refunded_customer_sk:   bigint?,
+    cr_refunded_cdemo_sk:      bigint?,
+    cr_refunded_hdemo_sk:      bigint?,
+    cr_refunded_addr_sk:       bigint?,
+    cr_returning_customer_sk:  bigint?,
+    cr_returning_cdemo_sk:     bigint?,
+    cr_returning_hdemo_sk:     bigint?,
+    cr_returning_addr_sk:      bigint?,
+    cr_call_center_sk:         bigint?,
+    cr_catalog_page_sk:        bigint?,
+    cr_ship_mode_sk:           bigint?,
+    cr_warehouse_sk:           bigint?,
+    cr_reason_sk:              bigint?,
+    cr_order_number:           bigint,
+    cr_return_quantity:        bigint?,
+    cr_return_amount:          double?,
+    cr_return_tax:             double?,
+    cr_return_amt_inc_tax:     double?,
+    cr_fee:                    double?,
+    cr_return_ship_cost:       double?,
+    cr_refunded_cash:          double?,
+    cr_reversed_charge:        double?,
+    cr_store_credit:           double?,
+    cr_net_loss:               double?
+};
+
+create type tpcds.item_type as closed {
+    i_item_sk:                 bigint,
+    i_item_id:                 string,
+    i_rec_start_date:          string?,
+    i_rec_end_date:            string?,
+    i_item_desc:               string?,
+    i_current_price:           double?,
+    i_wholesale_cost:          double?,
+    i_brand_id:                bigint? ,
+    i_brand:                   string?,
+    i_class_id:                bigint? ,
+    i_class:                   string?,
+    i_category_id:             bigint? ,
+    i_category:                string?,
+    i_manufact_id:             bigint? ,
+    i_manufact:                string?,
+    i_size:                    string?,
+    i_formulation:             string?,
+    i_color:                   string?,
+    i_units:                   string?,
+    i_container:               string?,
+    i_manager_id:              bigint?,
+    i_product_name:            string?
+};
+
+create dataset catalog_sales (catalog_sales_type) primary key cs_item_sk, cs_order_number;
+create dataset catalog_returns (catalog_returns_type) primary key cr_item_sk, cr_order_number;
+create dataset item (item_type) primary key i_item_sk;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT *
+FROM catalog_sales cs1
+     LEFT OUTER JOIN catalog_returns cr1
+     ON (cs1.cs_order_number = cr1.cr_order_number
+         AND cs1.cs_item_sk = cr1.cr_item_sk),
+     item i1
+WHERE i1.i_item_sk = cs1.cs_item_sk
+ORDER BY cs1.cs_item_sk, cs1.cs_order_number;
+
+drop dataverse tpcds;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp
new file mode 100644
index 0000000..6c55128
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP dataverse tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS {
+  l_linenumber : integer
+};
+
+CREATE TYPE OrderType AS {
+  o_orderkey : integer
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_linenumber;
+CREATE DATASET Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+/** The plan tests that the expression for different switch-case branches are not extracted.*/
+SELECT l.l_shipmode,
+       sum(CASE
+             WHEN o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH' THEN 1 +  o.o_orderpriority * 0
+             ELSE 0 + o.o_orderpriority * 0
+           END) high_line_count,
+       sum(CASE o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH'
+             WHEN true THEN 0 + o.o_orderpriority * 0
+             ELSE 1 +  o.o_orderpriority * 0
+           END) low_line_count
+FROM  LineItem l,
+      Orders o
+WHERE l.l_orderkey /*+ bcast */ = o.o_orderkey AND l.l_commitdate < l.l_receiptdate AND
+      l.l_shipdate < l.l_commitdate AND l.l_receiptdate >= '1994-01-01' AND
+      l.l_receiptdate < '1995-01-01' AND (l.l_shipmode = 'MAIL' OR l.l_shipmode = 'SHIP')
+GROUP BY l.l_shipmode
+ORDER BY l.l_shipmode;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp
new file mode 100644
index 0000000..62a9c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP dataverse tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS {
+  l_linenumber : integer
+};
+
+CREATE TYPE OrderType AS {
+  o_orderkey : integer
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_linenumber;
+CREATE DATASET Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+/** The plan tests that the expression for different switch-case branches are not extracted.*/
+SELECT l.l_shipmode,
+       sum(CASE
+             WHEN o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH' THEN 1 +  o.o_orderpriority * 0
+             ELSE 0 + o.o_orderpriority * 0
+           END) high_line_count,
+       sum(CASE o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH'
+             WHEN true THEN 0 + o.o_orderpriority * 0
+             ELSE 1 +  o.o_orderpriority * 0
+           END) low_line_count
+FROM  LineItem l,
+      Orders o
+WHERE o.o_orderkey = l.l_orderkey AND l.l_commitdate < l.l_receiptdate AND
+      l.l_shipdate < l.l_commitdate AND l.l_receiptdate >= '1994-01-01' AND
+      l.l_receiptdate < '1995-01-01' AND (l.l_shipmode = 'MAIL' OR l.l_shipmode = 'SHIP')
+GROUP BY l.l_shipmode
+ORDER BY l.l_shipmode;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
new file mode 100644
index 0000000..4966390
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -0,0 +1,109 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$64(ASC), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                              -- UNION_ALL  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SPLIT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$64(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$64]  |PARTITIONED|
+                                        -- UNION_ALL  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- SPLIT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
index 71ee3db..9594dc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
@@ -23,12 +23,10 @@
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -39,9 +37,7 @@
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
index 0ad315e..2f03f0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
@@ -24,19 +24,18 @@
                                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- STREAM_SELECT  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- BTREE_SEARCH  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -50,16 +49,15 @@
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH  |PARTITIONED|
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
new file mode 100644
index 0000000..7043901
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
@@ -0,0 +1,69 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- INTERSECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- INTERSECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
new file mode 100644
index 0000000..53589ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
new file mode 100644
index 0000000..7fe5795
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
@@ -0,0 +1,65 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- UNION_ALL  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- UNION_ALL  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- SPLIT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan
new file mode 100644
index 0000000..7763106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan
@@ -0,0 +1,117 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$33(ASC)] SPLIT COUNT:  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$29]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$29][$$32]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                              -- RANGE_PARTITION_EXCHANGE [$$32(ASC)] SPLIT COUNT:  |PARTITIONED|
+                                                -- FORWARD  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- REPLICATE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- AGGREGATE  |UNPARTITIONED|
+                                                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                        -- AGGREGATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- REPLICATE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$29]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$29][$$32]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                                      -- RANGE_PARTITION_EXCHANGE [$$32(ASC)] SPLIT COUNT:  |PARTITIONED|
+                                                        -- FORWARD  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                -- AGGREGATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- REPLICATE  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan
new file mode 100644
index 0000000..7262545
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$33(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$29]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$29][$$32]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$32(ASC)] HASH:[$$32]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..0a9089e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
new file mode 100644
index 0000000..173b7de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan
new file mode 100644
index 0000000..07563e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan
new file mode 100644
index 0000000..173b7de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan
new file mode 100644
index 0000000..07563e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
new file mode 100644
index 0000000..e061424
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$14(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
new file mode 100644
index 0000000..29b0fef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan
new file mode 100644
index 0000000..b9bd479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
new file mode 100644
index 0000000..f145eac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -0,0 +1,165 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- NESTED_LOOP  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
new file mode 100644
index 0000000..560158f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -0,0 +1,353 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$54][$$idRight_0]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                              -- EXTERNAL_GROUP_BY[$$135, $$133]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- HASH_PARTITION_EXCHANGE [$$135, $$133]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                              -- UNNEST  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
+                                                                          {
+                                                                            -- AGGREGATE  |LOCAL|
+                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                          }
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STABLE_SORT [$$70(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                {
+                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                }
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                            -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                    {
+                                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                    }
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                                                                              {
+                                                                                -- AGGREGATE  |LOCAL|
+                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                              }
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$64(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- UNNEST  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                  -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                              {
+                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                              }
+                                                                                                        -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                          -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                  {
+                                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                  }
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- UNNEST  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- NESTED_LOOP  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$54][$$idRight_0]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                        -- EXTERNAL_GROUP_BY[$$135, $$133]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- HASH_PARTITION_EXCHANGE [$$135, $$133]  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                        -- UNNEST  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
+                                                                                    {
+                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                        -- STREAM_SELECT  |LOCAL|
+                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                    }
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STABLE_SORT [$$70(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- UNNEST  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                              -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                          {
+                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                          }
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                                      -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- PRE_CLUSTERED_GROUP_BY[$$64]  |PARTITIONED|
+                                                                                        {
+                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                        }
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STABLE_SORT [$$64(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                            -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                        {
+                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                        }
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                                    -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
new file mode 100644
index 0000000..73103df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
@@ -0,0 +1,53 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$36(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$38][$$39]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
new file mode 100644
index 0000000..08b3940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
@@ -0,0 +1,53 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$29(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$61(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$69(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..9c29537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
new file mode 100644
index 0000000..575076b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan
new file mode 100644
index 0000000..da301ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan
new file mode 100644
index 0000000..575076b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan
new file mode 100644
index 0000000..da301ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
new file mode 100644
index 0000000..b4cb762
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$16(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
new file mode 100644
index 0000000..19649ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan
new file mode 100644
index 0000000..ae93a76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
new file mode 100644
index 0000000..9ed007f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -0,0 +1,190 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$86][$$65]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                        -- UNION_ALL  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- NESTED_LOOP  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$86][$$65]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                                  -- UNION_ALL  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- NESTED_LOOP  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
new file mode 100644
index 0000000..6ff2729
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -0,0 +1,358 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- NESTED_LOOP  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$66][$$idRight_0]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                -- EXTERNAL_GROUP_BY[$$162, $$160]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- HASH_PARTITION_EXCHANGE [$$162, $$160]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$93]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$93(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- UNNEST  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                      -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                  {
+                                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                  }
+                                                                                                            -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                              -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                      {
+                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                      }
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- UNNEST  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- PRE_CLUSTERED_GROUP_BY[$$87]  |PARTITIONED|
+                                                                                {
+                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                }
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STABLE_SORT [$$87(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                {
+                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                }
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                            -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                    {
+                                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                    }
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- UNNEST  |PARTITIONED|
+                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- NESTED_LOOP  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$66][$$idRight_0]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                          -- EXTERNAL_GROUP_BY[$$162, $$160]  |PARTITIONED|
+                                                                  {
+                                                                    -- AGGREGATE  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                  }
+                                                            -- HASH_PARTITION_EXCHANGE [$$162, $$160]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$93]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$93(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- UNNEST  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                                -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                                        -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                                {
+                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                }
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                              -- UNNEST  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- UNNEST  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$87]  |PARTITIONED|
+                                                                                          {
+                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                          }
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STABLE_SORT [$$87(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                              -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                          {
+                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                          }
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                                      -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
new file mode 100644
index 0000000..623c04d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$68(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$68(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- RTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
new file mode 100644
index 0000000..18c2244
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,92 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$77(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- RTREE_SEARCH  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$61(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$61(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$61(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$66(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$69(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$69(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$54]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..9c29537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
new file mode 100644
index 0000000..9ed007f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -0,0 +1,190 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$86][$$65]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                        -- UNION_ALL  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- REPLICATE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- NESTED_LOOP  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- REPLICATE  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$86][$$65]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                                  -- UNION_ALL  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- NESTED_LOOP  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
new file mode 100644
index 0000000..623c04d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$68(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$68(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- RTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
new file mode 100644
index 0000000..18c2244
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,92 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$77(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- RTREE_SEARCH  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$62]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$62(ASC), $$63(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
new file mode 100644
index 0000000..ccf2c9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
new file mode 100644
index 0000000..ccf2c9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$42]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$42(ASC), $$43(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$57(ASC)]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..0a9089e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
new file mode 100644
index 0000000..f145eac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -0,0 +1,165 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                      -- UNION_ALL  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- NESTED_LOOP  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- REPLICATE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$69][$$53]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                                -- UNION_ALL  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- NESTED_LOOP  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
new file mode 100644
index 0000000..c95c27f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$55(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$48(ASC), $$49(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
new file mode 100644
index 0000000..35e1ec6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$60(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan
new file mode 100644
index 0000000..101441e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.plan
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$30][$$31]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$30][$$31]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan
new file mode 100644
index 0000000..101441e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.plan
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$30][$$31]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$30][$$31]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
new file mode 100644
index 0000000..3da4637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan
new file mode 100644
index 0000000..2eb5d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan
new file mode 100644
index 0000000..2eb5d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
new file mode 100644
index 0000000..16c9194
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
new file mode 100644
index 0000000..16c9194
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
new file mode 100644
index 0000000..dec1cb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
@@ -0,0 +1,70 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$22(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- INTERSECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- INTERSECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
new file mode 100644
index 0000000..4adc026
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
new file mode 100644
index 0000000..93e872a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$name(DESC), $$age(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$name(DESC), $$age(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$39, $$40]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$39, $$40]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$39, $$40]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$39, $$40]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
new file mode 100644
index 0000000..2c7fcb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
new file mode 100644
index 0000000..386c64a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
new file mode 100644
index 0000000..2c7fcb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$21(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$28(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
new file mode 100644
index 0000000..386c64a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan
new file mode 100644
index 0000000..d5b2526
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$12(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan
new file mode 100644
index 0000000..32499ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
new file mode 100644
index 0000000..390afe3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -0,0 +1,74 @@
+-- SINK  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- BULKLOAD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$2(ASC), $$3(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2, $$3]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EXTERNAL_GROUP_BY[$$190, $$191]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$190, $$191]  |PARTITIONED|
+                          -- EXTERNAL_GROUP_BY[$$159, $$160]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EXTERNAL_GROUP_BY[$$190, $$191]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$190, $$191]  |PARTITIONED|
+                                    -- EXTERNAL_GROUP_BY[$$159, $$160]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
new file mode 100644
index 0000000..5585ddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
@@ -0,0 +1,59 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$149, $$150]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$149, $$150]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$117, $$118]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$149, $$150]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$149, $$150]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$117, $$118]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
new file mode 100644
index 0000000..a110303
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
@@ -0,0 +1,98 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$70(ASC), $$71(ASC), $$72(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$70(ASC), $$71(ASC), $$72(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- UNION_ALL  |PARTITIONED|
+                      -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- UNION_ALL  |PARTITIONED|
+                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- NESTED_LOOP  |PARTITIONED|
+                                              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
new file mode 100644
index 0000000..bd0ff12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
@@ -0,0 +1,72 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$39(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$37]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$37][$$40]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$37]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- STREAM_SELECT  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$37][$$40]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
new file mode 100644
index 0000000..ed8309c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$104, $$105]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$104(ASC), $$105(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$104, $$105]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$90, $$91]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$90(ASC), $$91(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$104, $$105]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$104(ASC), $$105(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$104, $$105]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$90, $$91]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$90(ASC), $$91(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
new file mode 100644
index 0000000..7b7285b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$112, $$113]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$112(ASC), $$113(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$99, $$100]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$99(ASC), $$100(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$112, $$113]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$112(ASC), $$113(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$99, $$100]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$99(ASC), $$100(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
new file mode 100644
index 0000000..8451f5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$120, $$121]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$108, $$109]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$108(ASC), $$109(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$120, $$121]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$108, $$109]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- STREAM_SELECT  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$108(ASC), $$109(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
new file mode 100644
index 0000000..b988a0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
@@ -0,0 +1,135 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$139]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$135]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$135(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$122][$$129]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$phone_substr][$$124]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$phone_substr]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$139]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$135]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$135(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$122][$$129]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- HYBRID_HASH_JOIN [$$phone_substr][$$124]  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- UNNEST  |UNPARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
index 6a471bc..07aed2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
@@ -32,12 +32,11 @@
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                         -- STREAM_SELECT  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- STREAM_SELECT  |PARTITIONED|
@@ -54,9 +53,8 @@
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- STREAM_SELECT  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
new file mode 100644
index 0000000..f8b50c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,133 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$55(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$88]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$88(ASC), $$49(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+                              -- UNION_ALL  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SPLIT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- RTREE_SEARCH  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SPLIT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$88]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$88(ASC), $$49(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+                                        -- UNION_ALL  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- SPLIT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- SPLIT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
new file mode 100644
index 0000000..35e1ec6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$60(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
index 008be34..45b3992 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
@@ -26,12 +26,10 @@
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- STREAM_SELECT  |PARTITIONED|
@@ -49,9 +47,7 @@
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |UNPARTITIONED|
+                  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
new file mode 100644
index 0000000..8361112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$79(ASC), $$80(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$79(ASC), $$80(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- NESTED_LOOP  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- REPLICATE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
new file mode 100644
index 0000000..4c9eff3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
@@ -0,0 +1,157 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$164]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$141]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$161]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$150][$$147]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |UNPARTITIONED|
+                                                                          -- ASSIGN  |UNPARTITIONED|
+                                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                                              -- AGGREGATE  |UNPARTITIONED|
+                                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                  -- AGGREGATE  |PARTITIONED|
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$164]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$141]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$161]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$150][$$147]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- NESTED_LOOP  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |UNPARTITIONED|
+                                                                                    -- ASSIGN  |UNPARTITIONED|
+                                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                            -- AGGREGATE  |PARTITIONED|
+                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
new file mode 100644
index 0000000..b4a8de2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
new file mode 100644
index 0000000..01a0df8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$22(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- UNNEST  |UNPARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
new file mode 100644
index 0000000..b4a8de2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+        -- FORWARD  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- REPLICATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |UNPARTITIONED|
+              -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                -- AGGREGATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
new file mode 100644
index 0000000..b430256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
@@ -0,0 +1,84 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$49(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$43]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- UNNEST  |LOCAL|
+                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[]  |LOCAL|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$43][$$42]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$43]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- UNNEST  |LOCAL|
+                                                    -- MICRO_PRE_CLUSTERED_GROUP_BY[]  |LOCAL|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- STREAM_SELECT  |LOCAL|
+                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$43][$$42]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
new file mode 100644
index 0000000..74bd5b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
@@ -0,0 +1,74 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$44(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$39]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$39][$$29]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$39]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$39][$$29]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
new file mode 100644
index 0000000..5a3e2f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
@@ -0,0 +1,157 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$cntrycode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$cntrycode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$165]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$162]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$151][$$148]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- NESTED_LOOP  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |UNPARTITIONED|
+                                                                          -- ASSIGN  |UNPARTITIONED|
+                                                                            -- AGGREGATE  |UNPARTITIONED|
+                                                                              -- AGGREGATE  |UNPARTITIONED|
+                                                                                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                  -- AGGREGATE  |PARTITIONED|
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$165]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$142]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$162]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$151][$$148]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- NESTED_LOOP  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |UNPARTITIONED|
+                                                                                    -- ASSIGN  |UNPARTITIONED|
+                                                                                      -- AGGREGATE  |UNPARTITIONED|
+                                                                                        -- AGGREGATE  |UNPARTITIONED|
+                                                                                          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                                                            -- AGGREGATE  |PARTITIONED|
+                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
new file mode 100644
index 0000000..9a0cc83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$44][$$48]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$45, $$44][$$47, $$46]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- AGGREGATE  |UNPARTITIONED|
+                      -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                        -- AGGREGATE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- REPLICATE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$44][$$48]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$45, $$44][$$47, $$46]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
new file mode 100644
index 0000000..17aa91c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$122]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$112][$$108]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$122]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$112][$$108]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
new file mode 100644
index 0000000..b7864dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$122]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$113][$$108]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$113]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$122]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$113][$$108]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$113]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
index 59a590e..42c2097 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
@@ -18,6 +18,8 @@
  */
 use dataverse testdv2;
 
+set "compiler.sortmemory" "32MB"
+
 for $d in dataset("testds")
-order by $d.id
+order by $d.name desc, $d.id
 return $d
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
index 77b930e..f0e735e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
@@ -28,7 +28,7 @@
 for $l in dataset('LineItem')
 where $l.l_orderkey>=0
       and $l.l_orderkey<100
-order by $l.l_linenumber
+order by $l.l_linenumber, $l.l_orderkey
 return {
   "l_linenumber": $l.l_linenumber,
   "l_l_orderkey": $l.l_orderkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
index a412072..a3ccb68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
@@ -25,5 +25,5 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey desc
+order by $i.l_partkey desc, $i.l_orderkey, $i.l_linenumber
 return $i
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
index 4bb5ca9..1c7faa6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
@@ -26,9 +26,9 @@
 for $i in dataset tsdata
   order by $i.time
   for $j in overlap-bins(interval-start-from-time($i.time, $i.duration), time("00:00:00"), day-time-duration("PT1H30M"))
-    group by $bin := $j with $i 
+    group by $bin := $j with $i
     order by get-interval-start($bin)
     for $x in $i
       let $itv := interval-start-from-time($x.time, $x.duration)
-      order by get-interval-start($bin)
+      order by get-interval-start($bin), $itv, get-overlapping-interval($bin, $itv)
       return { "tbin": $bin, "interval": $itv, "overlap": get-overlapping-interval($bin, $itv) }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp
new file mode 100644
index 0000000..d29e181
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing setting the number of samples for parallel sort
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp
similarity index 66%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp
index b92aa6c..8a78296 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp
@@ -16,18 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+use tpch;
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
-
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp
similarity index 66%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp
index b92aa6c..612b37a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp
@@ -16,18 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+USE tpch;
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+SET `compiler.sort.samples` "400";
 
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file
+SELECT VALUE v
+FROM LineItem v
+ORDER BY v.l_partkey, v.l_orderkey, v.l_linenumber;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp
similarity index 67%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp
index b92aa6c..af54590 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp
@@ -16,18 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+/*
+ * Description: testing a sequential merge when parallel sort has redistributed the data across partitions and one of
+ * the next operators requires merging the sorted data.
+ */
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+drop dataverse test if exists;
+create dataverse test;
 
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
+use test;
 
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
+create type TestType as
+{
+  id: int,
+  f1: int
+};
 
-}
\ No newline at end of file
+create  dataset TestDS(TestType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp
similarity index 66%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp
index b92aa6c..5aacecf 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp
@@ -16,18 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+use test;
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
-
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file
+INSERT INTO TestDS
+([
+{"id":1, "f1":40},
+{"id":2, "f1":40},
+{"id":3, "f1":40},
+{"id":4, "f1":41},
+{"id":5, "f1":42},
+{"id":6, "f1":42},
+{"id":7, "f1":40},
+{"id":8, "f1":41},
+{"id":9, "f1":41},
+{"id":10, "f1":42},
+{"id":11, "f1":40},
+{"id":12, "f1":41},
+{"id":13, "f1":42},
+{"id":14, "f1":41},
+{"id":15, "f1":40},
+{"id":16, "f1":42},
+{"id":17, "f1":42},
+{"id":18, "f1":41}
+]);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp
similarity index 66%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp
index b92aa6c..9c38f40 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp
@@ -16,18 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
 
-import java.io.DataInputStream;
+use test;
 
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+set `compiler.sort.parallel` "true";
 
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file
+[(select * from TestDS  v order by v.f1, v.id)];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
index 6837e2d..a397143 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
@@ -29,5 +29,5 @@
 select element {'l_linenumber':l.l_linenumber,'l_l_orderkey':l.l_orderkey}
 from  LineItem as l
 where ((l.l_orderkey >= 0) and (l.l_orderkey < 100))
-order by l.l_linenumber
+order by l.l_linenumber, l.l_orderkey
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
index dbcb4f0..baa27af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
@@ -24,8 +24,7 @@
 
 use test;
 
-
 select element i
 from  LineItem as i
-order by i.l_partkey desc
+order by i.l_partkey desc, i.l_orderkey, i.l_linenumber
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
index f7eabc3..f6f4a5c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
@@ -39,7 +39,7 @@
   "t5": (
     select value object_remove(u, "lang")
     from TwitterUsers as u
-    order by u.screen-name
+    order by u.`screen-name`
   ),
 
   /* closed type */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
index d8fda37..ee9b52d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
@@ -19,6 +19,8 @@
 
 use tpch;
 
+set `compiler.sort.parallel` "false";
+
 explain select value l
 from LineItem l
 where l_shipdate="1994-01-20"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
index d8fda37..ee9b52d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
@@ -19,6 +19,8 @@
 
 use tpch;
 
+set `compiler.sort.parallel` "false";
+
 explain select value l
 from LineItem l
 where l_shipdate="1994-01-20"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
index 07fa4fa..70a0e7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
@@ -35,4 +35,4 @@
     ) as gen0,
     gen0.i as x
 let itv = test.`interval-start-from-time`(x.time,x.duration)
-order by test.`get-interval-start`(gen0.bin);
+order by test.`get-interval-start`(gen0.bin), itv, test.`get-overlapping-interval`(gen0.bin,itv);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 79db9ae..050a799 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : 0,
+    "compiler\.sort\.parallel" : false,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 6e5547d..c56062a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : -1,
+    "compiler\.sort\.parallel" : true,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 3237551..3a3796d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : 3,
+    "compiler\.sort\.parallel" : true,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm
new file mode 100644
index 0000000..e43af35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm
@@ -0,0 +1,6005 @@
+{ "l_orderkey": 35, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ", regular tithe" }
+{ "l_orderkey": 134, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. quickly regular" }
+{ "l_orderkey": 321, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hockey players sleep slyly sl" }
+{ "l_orderkey": 548, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ideas. special accounts above the furiou" }
+{ "l_orderkey": 640, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36040.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oach according to the bol" }
+{ "l_orderkey": 807, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17119.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns haggle quickly across the furi" }
+{ "l_orderkey": 1122, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "t theodolites sleep. even, ironic" }
+{ "l_orderkey": 1154, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31535.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-30", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the carefully regular pinto beans boost" }
+{ "l_orderkey": 1287, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-08-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar packages. even, even" }
+{ "l_orderkey": 1472, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic theodolites hinder slyly slyly r" }
+{ "l_orderkey": 1668, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic requests. bold, final ideas a" }
+{ "l_orderkey": 1761, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ons boost fu" }
+{ "l_orderkey": 2117, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 24327.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the carefully ironic ideas" }
+{ "l_orderkey": 2374, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-26", "l_commitdate": "1993-12-15", "l_receiptdate": "1993-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending d" }
+{ "l_orderkey": 2528, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9010.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ely. fluffily even re" }
+{ "l_orderkey": 2534, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ideas. deposits use. slyly regular pa" }
+{ "l_orderkey": 2726, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-04", "l_commitdate": "1993-01-29", "l_receiptdate": "1993-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously bold theodolites" }
+{ "l_orderkey": 2883, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-26", "l_commitdate": "1995-03-04", "l_receiptdate": "1995-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s. final i" }
+{ "l_orderkey": 2885, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40545.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-10-30", "l_receiptdate": "1993-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ess ideas. regular, silen" }
+{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-10", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "the quickly even dolph" }
+{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lites sleep" }
+{ "l_orderkey": 3457, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tructions haggle alongsid" }
+{ "l_orderkey": 3843, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake. slyly even packages boost " }
+{ "l_orderkey": 3940, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thily. deposits cajole." }
+{ "l_orderkey": 4102, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-14", "l_commitdate": "1996-04-29", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even requests; regular pinto" }
+{ "l_orderkey": 4293, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30634.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ions sleep blithely on" }
+{ "l_orderkey": 4323, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-04", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "the slyly bold deposits slee" }
+{ "l_orderkey": 4355, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ought to mold. blithely pending ideas " }
+{ "l_orderkey": 4452, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42347.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. slyly regular cour" }
+{ "l_orderkey": 4580, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-13", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "requests. quickly silent asymptotes sle" }
+{ "l_orderkey": 5121, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1802.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-06-28", "l_receiptdate": "1992-08-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " final, regular account" }
+{ "l_orderkey": 5409, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8109.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-15", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " unusual, unusual reques" }
+{ "l_orderkey": 5634, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 901.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ctions haggle carefully. carefully clo" }
+{ "l_orderkey": 5760, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng the acco" }
+{ "l_orderkey": 5984, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7208.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its. express," }
+{ "l_orderkey": 65, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18942.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bove the even packages. accounts nag carefu" }
+{ "l_orderkey": 130, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43296.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely alongside of the regu" }
+{ "l_orderkey": 261, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30668.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "c packages. asymptotes da" }
+{ "l_orderkey": 290, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4510.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ans integrate. requests sleep. fur" }
+{ "l_orderkey": 418, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "regular, silent pinto" }
+{ "l_orderkey": 740, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "odolites cajole ironic, pending instruc" }
+{ "l_orderkey": 896, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6314.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-02", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " requests " }
+{ "l_orderkey": 1250, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, i" }
+{ "l_orderkey": 1575, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10824.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold accounts. furi" }
+{ "l_orderkey": 1696, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its maintain alongside of the f" }
+{ "l_orderkey": 2370, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-04-09", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "final depen" }
+{ "l_orderkey": 2437, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s deposits. pendi" }
+{ "l_orderkey": 2656, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts serve deposi" }
+{ "l_orderkey": 2662, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5412.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-30", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "olites cajole quickly along the b" }
+{ "l_orderkey": 3046, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 27962.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-30", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending somas alongside of the slyly iro" }
+{ "l_orderkey": 3362, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2706.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-26", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its cajole blithely excuses. de" }
+{ "l_orderkey": 3650, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-07-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re about the pinto " }
+{ "l_orderkey": 3654, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts doze bravely ab" }
+{ "l_orderkey": 3811, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 31570.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly final dolphins? quickly ironic frets" }
+{ "l_orderkey": 4001, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35178.0, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-13", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-06-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " dogged excuses. blithe" }
+{ "l_orderkey": 4032, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24354.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously according to" }
+{ "l_orderkey": 4135, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "posits cajole furiously carefully" }
+{ "l_orderkey": 4387, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s hinder quietly across the pla" }
+{ "l_orderkey": 4389, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "at the final excuses hinder carefully a" }
+{ "l_orderkey": 4454, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "equests run." }
+{ "l_orderkey": 5090, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-03", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ular requests su" }
+{ "l_orderkey": 5312, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38786.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-03-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly unusual" }
+{ "l_orderkey": 5348, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-02-02", "l_receiptdate": "1997-12-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y according to the carefully pending acco" }
+{ "l_orderkey": 5478, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42394.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " instructions; slyly even accounts hagg" }
+{ "l_orderkey": 5699, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "kages. fin" }
+{ "l_orderkey": 5828, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25256.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special ideas haggle slyly ac" }
+{ "l_orderkey": 5862, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26158.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e fluffily. furiously" }
+{ "l_orderkey": 5893, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1804.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-18", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-08-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages wake sly" }
+{ "l_orderkey": 5957, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15334.0, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final, pending packages" }
+{ "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 39, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 39732.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eodolites. careful" }
+{ "l_orderkey": 129, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41538.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-01-24", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uietly bold theodolites. fluffil" }
+{ "l_orderkey": 194, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular deposi" }
+{ "l_orderkey": 519, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34314.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1997-12-15", "l_receiptdate": "1998-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular excuses detect quickly furiously " }
+{ "l_orderkey": 801, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18963.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cial, special packages." }
+{ "l_orderkey": 993, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lites. even theodolite" }
+{ "l_orderkey": 999, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9030.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-23", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully pending" }
+{ "l_orderkey": 1186, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily spec" }
+{ "l_orderkey": 1508, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cording to the furiously ironic depe" }
+{ "l_orderkey": 1542, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 10836.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "carefully " }
+{ "l_orderkey": 2372, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xcuses. slyly ironic theod" }
+{ "l_orderkey": 2401, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 44247.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "lites cajole carefully " }
+{ "l_orderkey": 2721, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1806.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-13", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " slyly final requests against " }
+{ "l_orderkey": 2946, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31605.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-15", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sublate along the fluffily iron" }
+{ "l_orderkey": 2951, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans wake ac" }
+{ "l_orderkey": 3015, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " the furiously pendi" }
+{ "l_orderkey": 3110, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 30702.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly pending requests ha" }
+{ "l_orderkey": 3137, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5418.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-10-23", "l_receiptdate": "1995-10-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly express as" }
+{ "l_orderkey": 3331, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23478.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-17", "l_receiptdate": "1993-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "p asymptotes. carefully unusual in" }
+{ "l_orderkey": 3776, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35217.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "yly blithely pending packages" }
+{ "l_orderkey": 3937, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26187.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-22", "l_receiptdate": "1998-03-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nt pinto beans above the pending instr" }
+{ "l_orderkey": 4484, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 37926.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding, pending requests wake. fluffily " }
+{ "l_orderkey": 4740, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19866.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-08-17", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final dependencies nag " }
+{ "l_orderkey": 5252, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37023.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ording to the blithely express somas sho" }
+{ "l_orderkey": 5892, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ithely unusual accounts will have to integ" }
+{ "l_orderkey": 164, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 20792.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ress packages haggle ideas. blithely spec" }
+{ "l_orderkey": 641, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37064.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-27", "l_receiptdate": "1993-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " asymptotes are quickly. bol" }
+{ "l_orderkey": 704, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the quickly final forges. furiously p" }
+{ "l_orderkey": 739, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 45200.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-26", "l_commitdate": "1998-07-16", "l_receiptdate": "1998-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ndencies. blith" }
+{ "l_orderkey": 1251, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33448.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-21", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". furiously" }
+{ "l_orderkey": 1316, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6328.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". furiously even accounts a" }
+{ "l_orderkey": 2019, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28024.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l ideas across the slowl" }
+{ "l_orderkey": 2144, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-05-16", "l_receiptdate": "1994-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully ironic" }
+{ "l_orderkey": 2279, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing foxes above the even accounts use slyly" }
+{ "l_orderkey": 2404, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16272.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-25", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-07-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "packages. even requests according to " }
+{ "l_orderkey": 2560, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24408.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " against the carefully" }
+{ "l_orderkey": 2593, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the furiously " }
+{ "l_orderkey": 2790, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28928.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-25", "l_commitdate": "1994-10-26", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ully pending" }
+{ "l_orderkey": 2882, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kly. even requests w" }
+{ "l_orderkey": 2885, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-12-12", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions solve. slyly regular requests n" }
+{ "l_orderkey": 2976, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21696.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic pinto beans. slyly bol" }
+{ "l_orderkey": 3522, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1994-12-09", "l_receiptdate": "1995-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes snooze " }
+{ "l_orderkey": 4005, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23504.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1997-02-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " to the quic" }
+{ "l_orderkey": 4099, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slowly final warthogs sleep blithely. q" }
+{ "l_orderkey": 4196, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular packages haggle furiously alongs" }
+{ "l_orderkey": 4292, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 42488.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y packages; even ideas boost" }
+{ "l_orderkey": 4736, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38872.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1995-12-21", "l_receiptdate": "1996-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. carefully " }
+{ "l_orderkey": 5349, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-10-08", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits affix carefully" }
+{ "l_orderkey": 5411, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17176.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ial accounts according to the f" }
+{ "l_orderkey": 5668, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13560.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the express, pending requests. bo" }
+{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 904.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 228, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2715.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckages. sly" }
+{ "l_orderkey": 320, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27150.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic, final accounts wake quick de" }
+{ "l_orderkey": 354, "l_partkey": 5, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t thinly above the ironic, " }
+{ "l_orderkey": 548, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5430.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits wake furiously regular" }
+{ "l_orderkey": 645, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 38915.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously accounts. slyly" }
+{ "l_orderkey": 675, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits along the express foxes " }
+{ "l_orderkey": 966, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial ins" }
+{ "l_orderkey": 993, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "sits. pending pinto beans haggle? ca" }
+{ "l_orderkey": 1058, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the final requests believe carefully " }
+{ "l_orderkey": 1155, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44345.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-12-30", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts are alongside of t" }
+{ "l_orderkey": 1574, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 38010.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans according t" }
+{ "l_orderkey": 1633, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-13", "l_commitdate": "1995-11-13", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ges wake fluffil" }
+{ "l_orderkey": 1637, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " haggle carefully silent accou" }
+{ "l_orderkey": 1732, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45250.0, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-23", "l_receiptdate": "1993-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily final asymptotes according " }
+{ "l_orderkey": 1829, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9955.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding orbits" }
+{ "l_orderkey": 2178, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36200.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes are slowly regularly specia" }
+{ "l_orderkey": 2241, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " final deposits use fluffily. even f" }
+{ "l_orderkey": 2375, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4525.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "final packages cajole according to the furi" }
+{ "l_orderkey": 2725, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns sleep furiously c" }
+{ "l_orderkey": 2819, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25340.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages sublate carefully closely regular " }
+{ "l_orderkey": 3076, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 28055.0, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular depos" }
+{ "l_orderkey": 3328, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 20815.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-12", "l_commitdate": "1993-02-07", "l_receiptdate": "1993-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. careful" }
+{ "l_orderkey": 3555, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17195.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep special theodolit" }
+{ "l_orderkey": 3649, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "special re" }
+{ "l_orderkey": 3680, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-01-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "iously ironic platelets in" }
+{ "l_orderkey": 3877, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-30", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "integrate against the expres" }
+{ "l_orderkey": 3970, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-05-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ix slyly. quickly silen" }
+{ "l_orderkey": 4229, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30770.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-04-13", "l_receiptdate": "1998-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thely final accounts use even packa" }
+{ "l_orderkey": 5283, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-16", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al deposits? blithely even pinto beans" }
+{ "l_orderkey": 5665, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "- special pinto beans sleep quickly blithel" }
+{ "l_orderkey": 5959, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3620.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-07-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests ar" }
+{ "l_orderkey": 260, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26274.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fluffily even asymptotes. express wa" }
+{ "l_orderkey": 290, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 31710.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-04-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ove the final foxes detect slyly fluffily" }
+{ "l_orderkey": 768, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27180.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously fluffy pinto beans haggle along" }
+{ "l_orderkey": 801, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are fluffily stealthily expres" }
+{ "l_orderkey": 1124, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "t the slyly " }
+{ "l_orderkey": 1220, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32616.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unusual, silent pinto beans aga" }
+{ "l_orderkey": 1284, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " regular asymptotes. " }
+{ "l_orderkey": 1542, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending instr" }
+{ "l_orderkey": 1638, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-10-28", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "otes haggle before the slyly bold instructi" }
+{ "l_orderkey": 1827, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-17", "l_commitdate": "1996-08-29", "l_receiptdate": "1996-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely. express, bo" }
+{ "l_orderkey": 2049, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35334.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the even pinto beans " }
+{ "l_orderkey": 2054, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n pinto beans. ironic courts are iro" }
+{ "l_orderkey": 2179, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " cajole carefully. " }
+{ "l_orderkey": 2276, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-30", "l_receiptdate": "1996-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. deposits " }
+{ "l_orderkey": 2370, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19026.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ecial dependencies must have to " }
+{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40770.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
+{ "l_orderkey": 3043, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13590.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "usly furiously" }
+{ "l_orderkey": 3137, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. silent excuses boost about" }
+{ "l_orderkey": 3329, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final depo" }
+{ "l_orderkey": 3426, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pecial theodolites haggle fluf" }
+{ "l_orderkey": 4005, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld requests. slyly final instructi" }
+{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
+{ "l_orderkey": 4195, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. carefully express" }
+{ "l_orderkey": 4387, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the blithely regular fox" }
+{ "l_orderkey": 4483, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 28992.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests haggle. slyl" }
+{ "l_orderkey": 4485, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 42582.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffily pending acc" }
+{ "l_orderkey": 4612, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18120.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans sleep blithely iro" }
+{ "l_orderkey": 4676, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 29898.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-29", "l_commitdate": "1995-10-01", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly express " }
+{ "l_orderkey": 4870, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its wake quickly. slyly quick" }
+{ "l_orderkey": 5124, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-05", "l_commitdate": "1997-06-29", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "wake across the" }
+{ "l_orderkey": 5125, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ily even deposits w" }
+{ "l_orderkey": 5223, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17214.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-28", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntly. furiously even excuses a" }
+{ "l_orderkey": 5760, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s. bravely ironic accounts among" }
+{ "l_orderkey": 5957, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es across the regular requests maint" }
+{ "l_orderkey": 579, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37187.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold, express requests sublate slyly. blith" }
+{ "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-18", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully. pending in" }
+{ "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 12698.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-31", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r, final packages are slyly iro" }
+{ "l_orderkey": 834, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9977.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inst the regular packa" }
+{ "l_orderkey": 1281, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-02-02", "l_receiptdate": "1995-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ounts detect" }
+{ "l_orderkey": 2080, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4535.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-26", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "refully unusual theo" }
+{ "l_orderkey": 2150, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "press platelets haggle until the slyly fi" }
+{ "l_orderkey": 2182, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ments are fu" }
+{ "l_orderkey": 2208, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-05-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e fluffily regular theodolites caj" }
+{ "l_orderkey": 2598, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-17", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "express packages nag sly" }
+{ "l_orderkey": 2658, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-02", "l_commitdate": "1995-11-08", "l_receiptdate": "1995-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e special requests. quickly ex" }
+{ "l_orderkey": 2659, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8163.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-07", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly final packages sleep ac" }
+{ "l_orderkey": 2752, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26303.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gly blithely re" }
+{ "l_orderkey": 3140, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19047.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furiously sly excuses according to the" }
+{ "l_orderkey": 3204, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35373.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-19", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sits sleep theodolites. slyly bo" }
+{ "l_orderkey": 4166, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15419.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages. re" }
+{ "l_orderkey": 4614, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 17233.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-06-21", "l_receiptdate": "1996-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ix. carefully regular " }
+{ "l_orderkey": 5318, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ickly final deposi" }
+{ "l_orderkey": 5351, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss the ironic, regular asymptotes cajole " }
+{ "l_orderkey": 5606, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22675.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "breach about the furiously bold " }
+{ "l_orderkey": 5670, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21768.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "press, express requests haggle" }
+{ "l_orderkey": 5794, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13605.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-27", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular ideas. final foxes haggle " }
+{ "l_orderkey": 68, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully special instructions cajole. furious" }
+{ "l_orderkey": 225, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28148.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "special platelets. quickly r" }
+{ "l_orderkey": 230, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 908.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely unusual dolphins. bold, ex" }
+{ "l_orderkey": 1060, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-12", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts; even deposits are carefull" }
+{ "l_orderkey": 1222, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-13", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ", even accounts are ironic" }
+{ "l_orderkey": 1283, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 27240.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-22", "l_receiptdate": "1996-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t the fluffily" }
+{ "l_orderkey": 1472, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "riously silent deposits to the pending d" }
+{ "l_orderkey": 1540, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-15", "l_commitdate": "1992-10-24", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ironic deposits amo" }
+{ "l_orderkey": 1760, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly bold dolphins haggle carefully. sl" }
+{ "l_orderkey": 1762, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44492.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-20", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages sleep fluffily pen" }
+{ "l_orderkey": 1828, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " accounts run slyly " }
+{ "l_orderkey": 2048, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4540.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "affix carefully against " }
+{ "l_orderkey": 2277, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1816.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "endencies sleep idly pending p" }
+{ "l_orderkey": 3777, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9080.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le. ironic depths a" }
+{ "l_orderkey": 4291, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uctions. furiously regular ins" }
+{ "l_orderkey": 4322, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10896.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e blithely against the slyly unusu" }
+{ "l_orderkey": 4420, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6356.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular instructions sleep around" }
+{ "l_orderkey": 4613, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-22", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gainst the furiously ironic" }
+{ "l_orderkey": 4614, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-26", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic foxes affix furi" }
+{ "l_orderkey": 5057, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-10-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " asymptotes wake slyl" }
+{ "l_orderkey": 5478, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35412.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-09-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. furiously " }
+{ "l_orderkey": 5600, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17252.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dencies. carefully p" }
+{ "l_orderkey": 5635, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36320.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending foxes. regular packages" }
+{ "l_orderkey": 5894, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20884.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-09-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " furiously even deposits haggle alw" }
+{ "l_orderkey": 293, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12726.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es. packages above the" }
+{ "l_orderkey": 419, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13635.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "of the careful, thin theodolites. quickly s" }
+{ "l_orderkey": 903, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 31815.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "rets wake fin" }
+{ "l_orderkey": 903, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 29997.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely ironic packages wake blithely" }
+{ "l_orderkey": 1152, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20907.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "equests alongside of the unusual " }
+{ "l_orderkey": 1568, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 41814.0, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-04-08", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "g the blithely even acco" }
+{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4545.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely regular accounts are slyly. pending, bo" }
+{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nts. fluffily special instructions integr" }
+{ "l_orderkey": 1955, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14544.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "odolites eat s" }
+{ "l_orderkey": 2055, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13635.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-21", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular foxes. b" }
+{ "l_orderkey": 2693, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23634.0, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-14", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "cajole alo" }
+{ "l_orderkey": 2976, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29088.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nding, ironic deposits sleep f" }
+{ "l_orderkey": 2979, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "st blithely; blithely regular gifts dazz" }
+{ "l_orderkey": 3075, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35451.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-06-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ing deposits nag " }
+{ "l_orderkey": 3233, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22725.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oss the pl" }
+{ "l_orderkey": 3970, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 41814.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-05-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly ironic" }
+{ "l_orderkey": 4096, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes mold flu" }
+{ "l_orderkey": 4196, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28179.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the blithely ironic inst" }
+{ "l_orderkey": 4199, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16362.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-06-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "pending, regular accounts. carefully" }
+{ "l_orderkey": 4416, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40905.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the final pinto beans. special frets " }
+{ "l_orderkey": 4483, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45450.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. furiously ironi" }
+{ "l_orderkey": 4545, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 32724.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sublate slyly. furiously ironic accounts b" }
+{ "l_orderkey": 4771, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-22", "l_receiptdate": "1992-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " carefully re" }
+{ "l_orderkey": 4805, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38178.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-07-03", "l_receiptdate": "1992-09-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the regular, fina" }
+{ "l_orderkey": 5221, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30906.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-07-17", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eans. furio" }
+{ "l_orderkey": 5414, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17271.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ffily silent theodolites na" }
+{ "l_orderkey": 5511, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 20907.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ing dugouts " }
+{ "l_orderkey": 5536, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27270.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-05-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "arefully regular theodolites according" }
+{ "l_orderkey": 5859, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15453.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic requests. quickly unusual pin" }
+{ "l_orderkey": 196, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13650.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s accounts. furio" }
+{ "l_orderkey": 230, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 40040.44, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-03-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "deposits integrate slyly sile" }
+{ "l_orderkey": 449, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2730.03, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-28", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " bold deposits. express theodolites haggle" }
+{ "l_orderkey": 519, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 11830.13, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-06", "l_commitdate": "1997-12-02", "l_receiptdate": "1998-03-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c accounts wake along the ironic so" }
+{ "l_orderkey": 994, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10010.11, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-05-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accounts sleep " }
+{ "l_orderkey": 998, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20020.22, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-02-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lites. qui" }
+{ "l_orderkey": 1382, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-17", "l_commitdate": "1993-09-28", "l_receiptdate": "1993-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake pending pinto beans. s" }
+{ "l_orderkey": 1668, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep slyly across the furi" }
+{ "l_orderkey": 1730, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36400.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-02", "l_commitdate": "1998-10-06", "l_receiptdate": "1998-10-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ven dinos slee" }
+{ "l_orderkey": 1796, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly ironic accounts." }
+{ "l_orderkey": 1922, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11830.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-11-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quests. furiously" }
+{ "l_orderkey": 1989, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 42770.47, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-21", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "final deposits s" }
+{ "l_orderkey": 2881, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 910.01, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "final theodolites. quickly" }
+{ "l_orderkey": 2946, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22750.25, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-06", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic deposits. furiously" }
+{ "l_orderkey": 2947, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33670.37, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e accounts: expres" }
+{ "l_orderkey": 2976, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31850.35, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "boost slyly about the regular, regular re" }
+{ "l_orderkey": 2980, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43680.48, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "totes. regular pinto " }
+{ "l_orderkey": 3138, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 10920.12, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ". bold pinto beans haggl" }
+{ "l_orderkey": 3141, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33670.37, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "press pinto beans. bold accounts boost b" }
+{ "l_orderkey": 3363, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38220.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " blithely final ideas nag after" }
+{ "l_orderkey": 3558, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l deposits " }
+{ "l_orderkey": 3941, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1820.02, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-12-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "es wake after the" }
+{ "l_orderkey": 4102, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15470.17, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly silent theodolites sleep unusual exc" }
+{ "l_orderkey": 4161, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40950.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "about the ironic packages cajole blithe" }
+{ "l_orderkey": 4807, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37310.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-31", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " fluffily re" }
+{ "l_orderkey": 4839, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22750.25, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-07-08", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "regular packages ab" }
+{ "l_orderkey": 4935, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12740.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slowly. blith" }
+{ "l_orderkey": 4999, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40040.44, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ependencies. slowly regu" }
+{ "l_orderkey": 5378, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16380.18, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-21", "l_receiptdate": "1992-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic accounts was bold, " }
+{ "l_orderkey": 5507, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20930.23, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-07-04", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ously slow packages poach whithout the" }
+{ "l_orderkey": 5632, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 43680.48, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-03-24", "l_receiptdate": "1996-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "unts. decoys u" }
+{ "l_orderkey": 103, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33707.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ies. quickly ironic requests use blithely" }
+{ "l_orderkey": 198, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31885.35, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests nod quickly furiously sly pinto be" }
+{ "l_orderkey": 773, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28241.31, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e slyly unusual deposit" }
+{ "l_orderkey": 928, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "longside of" }
+{ "l_orderkey": 998, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5466.06, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1994-12-27", "l_receiptdate": "1995-04-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "refully accounts. carefully express ac" }
+{ "l_orderkey": 1509, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously regula" }
+{ "l_orderkey": 2054, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 3644.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-12", "l_commitdate": "1992-08-31", "l_receiptdate": "1992-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lyly careful requests wake fl" }
+{ "l_orderkey": 2147, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " the fluffily" }
+{ "l_orderkey": 2466, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26419.29, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es boost fluffily ab" }
+{ "l_orderkey": 2469, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 43728.48, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "riously even theodolites u" }
+{ "l_orderkey": 2979, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42817.47, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-25", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously unusual dependencies wake across" }
+{ "l_orderkey": 3237, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-31", "l_receiptdate": "1992-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es. permanently express platelets besid" }
+{ "l_orderkey": 3460, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36440.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "odolites are slyly bold deposits" }
+{ "l_orderkey": 3584, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3644.04, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nal packag" }
+{ "l_orderkey": 3872, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37351.41, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly regular epitaphs boost" }
+{ "l_orderkey": 4192, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32796.36, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-25", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eodolites sleep" }
+{ "l_orderkey": 4263, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34618.38, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-07-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rding to the dep" }
+{ "l_orderkey": 4389, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20042.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lly silent de" }
+{ "l_orderkey": 4551, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5466.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily silent fo" }
+{ "l_orderkey": 4800, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19131.21, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-15", "l_receiptdate": "1992-02-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ithely according to " }
+{ "l_orderkey": 4866, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8199.09, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-18", "l_receiptdate": "1997-09-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven dependencies x-ray. quic" }
+{ "l_orderkey": 4934, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1822.02, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ongside of the brave, regula" }
+{ "l_orderkey": 4935, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21864.24, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-06-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly quickly s" }
+{ "l_orderkey": 5187, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 44639.49, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-20", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, regular platelets instead of the foxes w" }
+{ "l_orderkey": 5573, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s haggle qu" }
+{ "l_orderkey": 5633, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 35529.39, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ding ideas cajole furiously after" }
+{ "l_orderkey": 5698, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27330.3, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its. quickly regular foxes aro" }
+{ "l_orderkey": 5858, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 45550.5, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-07-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "r the ironic ex" }
+{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
+{ "l_orderkey": 130, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " slyly ironic decoys abou" }
+{ "l_orderkey": 322, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 31920.35, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-05-03", "l_receiptdate": "1992-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular accounts cajole carefully. even d" }
+{ "l_orderkey": 359, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "unusual warthogs. ironically sp" }
+{ "l_orderkey": 807, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10032.11, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts above the slyly final ex" }
+{ "l_orderkey": 928, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 34656.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "xpress grouc" }
+{ "l_orderkey": 1123, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9120.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-11-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckages are above the depths. slyly ir" }
+{ "l_orderkey": 1156, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26448.29, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts sleep sly" }
+{ "l_orderkey": 1345, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-12-11", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly express requests. ironic accounts c" }
+{ "l_orderkey": 1763, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20064.22, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-15", "l_receiptdate": "1997-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ld. fluffily final ideas boos" }
+{ "l_orderkey": 1797, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19152.21, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-05", "l_commitdate": "1996-08-05", "l_receiptdate": "1996-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ns. regular, regular deposit" }
+{ "l_orderkey": 2274, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-12-03", "l_receiptdate": "1993-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "usly final re" }
+{ "l_orderkey": 2311, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29184.32, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-19", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sts along the slyly" }
+{ "l_orderkey": 2435, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21888.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. carefully regular d" }
+{ "l_orderkey": 2497, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31008.34, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic accounts. p" }
+{ "l_orderkey": 3204, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9120.1, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-08", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "counts. bold " }
+{ "l_orderkey": 3239, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28272.31, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "foxes. pendin" }
+{ "l_orderkey": 3715, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-04-30", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ut the carefully expr" }
+{ "l_orderkey": 4038, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " packages " }
+{ "l_orderkey": 4771, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4560.05, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1993-01-19", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar, quiet accounts nag furiously express id" }
+{ "l_orderkey": 5636, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 30096.33, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-04-05", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ding to the " }
+{ "l_orderkey": 5729, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45600.5, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-12-31", "l_receiptdate": "1994-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly special sentiments. car" }
+{ "l_orderkey": 5762, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10944.12, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages are abo" }
+{ "l_orderkey": 5920, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25536.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-13", "l_receiptdate": "1994-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le slyly slyly even deposits. f" }
+{ "l_orderkey": 37, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 39259.43, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously ste" }
+{ "l_orderkey": 322, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18260.2, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-26", "l_commitdate": "1992-05-04", "l_receiptdate": "1992-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ckly toward " }
+{ "l_orderkey": 514, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5478.06, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-06-04", "l_receiptdate": "1996-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "as haggle blithely; quickly s" }
+{ "l_orderkey": 579, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25564.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-05-24", "l_receiptdate": "1998-07-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ic ideas until th" }
+{ "l_orderkey": 643, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25564.28, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-13", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly regular requests nag sly" }
+{ "l_orderkey": 933, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24651.27, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ests. express" }
+{ "l_orderkey": 935, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7304.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-11-02", "l_receiptdate": "1998-02-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cept the quickly regular p" }
+{ "l_orderkey": 1379, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21912.24, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-06", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ages cajole carefully idly express re" }
+{ "l_orderkey": 1537, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40172.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar courts." }
+{ "l_orderkey": 1634, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 31955.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-11-25", "l_receiptdate": "1996-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cies. regular, special de" }
+{ "l_orderkey": 1828, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36520.4, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s use above the quietly fin" }
+{ "l_orderkey": 2081, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 29216.32, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-09-26", "l_receiptdate": "1997-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e. final, regular dependencies sleep slyly!" }
+{ "l_orderkey": 2432, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-18", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "riously regular packages. p" }
+{ "l_orderkey": 2438, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28303.31, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t. slyly ironic sh" }
+{ "l_orderkey": 2723, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 42911.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "furiously r" }
+{ "l_orderkey": 2753, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5478.06, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s accounts" }
+{ "l_orderkey": 3239, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11869.13, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "r deposits solve fluf" }
+{ "l_orderkey": 3397, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10043.11, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "iously careful packages. s" }
+{ "l_orderkey": 3648, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14608.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously stealthy deposits haggle furi" }
+{ "l_orderkey": 4674, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19173.21, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ent accounts sublate deposits. instruc" }
+{ "l_orderkey": 4965, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22825.25, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1993-12-15", "l_receiptdate": "1994-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "wake at the carefully speci" }
+{ "l_orderkey": 5313, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15521.17, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uests wake" }
+{ "l_orderkey": 5445, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-19", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests. bravely i" }
+{ "l_orderkey": 5763, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22825.25, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-08-16", "l_receiptdate": "1998-10-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inal theodolites. even re" }
+{ "l_orderkey": 5831, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 41998.46, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-01-18", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly final pa" }
+{ "l_orderkey": 5953, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31042.34, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-06-12", "l_receiptdate": "1992-06-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hockey players use furiously against th" }
+{ "l_orderkey": 453, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34732.38, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-10", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sts cajole. furiously un" }
+{ "l_orderkey": 899, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10054.11, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t the ironic" }
+{ "l_orderkey": 1510, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2742.03, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "along the slyly regular pin" }
+{ "l_orderkey": 1858, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30162.33, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-01-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tect along the slyly final" }
+{ "l_orderkey": 1953, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 31990.35, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-02-25", "l_receiptdate": "1994-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "among the fur" }
+{ "l_orderkey": 2020, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27420.3, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-08", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly about the blithely ironic foxes. bold" }
+{ "l_orderkey": 2279, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10968.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-10", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lets across the excuses nag quickl" }
+{ "l_orderkey": 2368, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29248.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-23", "l_commitdate": "1993-10-07", "l_receiptdate": "1993-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gular courts use blithely around the" }
+{ "l_orderkey": 2466, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26506.29, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-11", "l_commitdate": "1994-04-27", "l_receiptdate": "1994-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckages. bold requests nag carefully." }
+{ "l_orderkey": 2981, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15538.17, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ", unusual packages x-ray. furious" }
+{ "l_orderkey": 3042, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1994-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "can wake after the enticingly stealthy i" }
+{ "l_orderkey": 3047, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21022.23, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly ironi" }
+{ "l_orderkey": 3232, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20108.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-12-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "thely. furio" }
+{ "l_orderkey": 3399, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "se final courts. exc" }
+{ "l_orderkey": 3425, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7312.08, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-07-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously regular theodolites wake. s" }
+{ "l_orderkey": 3426, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17366.19, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-02", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly special packages oug" }
+{ "l_orderkey": 3751, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35646.39, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-16", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "refully according to the iro" }
+{ "l_orderkey": 3781, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 43872.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-22", "l_commitdate": "1996-08-13", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "equests may cajole careful" }
+{ "l_orderkey": 4322, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ounts haggle fluffily ideas. pend" }
+{ "l_orderkey": 4548, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pecial theodoli" }
+{ "l_orderkey": 4929, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18280.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final pinto beans detect. final," }
+{ "l_orderkey": 5028, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13710.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es are quickly final pains. furiously pend" }
+{ "l_orderkey": 5253, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8226.09, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly express deposits use furiou" }
+{ "l_orderkey": 5731, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5484.06, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rs. quickly regular theo" }
+{ "l_orderkey": 5792, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 12796.14, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-06-17", "l_receiptdate": "1993-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "olites print carefully" }
+{ "l_orderkey": 160, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32940.36, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-11", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "old, ironic deposits are quickly abov" }
+{ "l_orderkey": 774, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7320.08, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-15", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ully ironic requests c" }
+{ "l_orderkey": 1474, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4575.05, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-22", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ully final a" }
+{ "l_orderkey": 2151, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26535.29, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " bold packages acro" }
+{ "l_orderkey": 2309, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4575.05, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-29", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. requests wake blithely specia" }
+{ "l_orderkey": 2563, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38430.42, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ymptotes nag furiously slyly even inst" }
+{ "l_orderkey": 2688, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42090.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "elets. regular reque" }
+{ "l_orderkey": 2913, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 11895.13, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inos are carefully alongside of the bol" }
+{ "l_orderkey": 3109, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9150.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits haggle carefully. regular, unusual ac" }
+{ "l_orderkey": 3843, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6405.07, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly even instructions. furiously eve" }
+{ "l_orderkey": 3846, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-03-22", "l_receiptdate": "1998-02-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "efully even packages against the blithe" }
+{ "l_orderkey": 4005, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tions sleep across the silent d" }
+{ "l_orderkey": 4065, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42090.46, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ", regular requests may mold above the " }
+{ "l_orderkey": 4132, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21045.23, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-27", "l_commitdate": "1995-07-27", "l_receiptdate": "1995-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "d deposits. fluffily even requests haggle b" }
+{ "l_orderkey": 4354, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27450.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1994-11-24", "l_receiptdate": "1995-02-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "around the ir" }
+{ "l_orderkey": 4450, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8235.09, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-13", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "gular requests cajole carefully. regular c" }
+{ "l_orderkey": 4832, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21045.23, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1998-01-05", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y express depo" }
+{ "l_orderkey": 5472, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 915.01, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s use furiou" }
+{ "l_orderkey": 5473, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 30195.33, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-06-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "efully above the even, " }
+{ "l_orderkey": 5895, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34770.38, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts are furiously. regular, final excuses " }
+{ "l_orderkey": 5957, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33855.37, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ideas use ruthlessly." }
+{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+{ "l_orderkey": 198, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18320.2, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully final escapades a" }
+{ "l_orderkey": 295, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-11-17", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final instructions h" }
+{ "l_orderkey": 420, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 35724.39, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. ironic waters about the car" }
+{ "l_orderkey": 1315, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13740.15, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". foxes integrate carefully special" }
+{ "l_orderkey": 1346, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 41220.45, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "press deposits." }
+{ "l_orderkey": 1696, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-05-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the blithely" }
+{ "l_orderkey": 1861, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1832.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e final, regular requests. carefully " }
+{ "l_orderkey": 1987, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6412.07, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular a" }
+{ "l_orderkey": 2178, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24732.27, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across the ironic reques" }
+{ "l_orderkey": 2180, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28396.31, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-11-21", "l_receiptdate": "1996-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "n requests are furiously at the quickly" }
+{ "l_orderkey": 2789, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 43052.47, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "usly busy packages wake against the unusual" }
+{ "l_orderkey": 3040, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16488.18, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly thin accou" }
+{ "l_orderkey": 3365, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1832.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "es cajole fluffily pe" }
+{ "l_orderkey": 3458, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14656.16, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s grow carefully. express, final grouc" }
+{ "l_orderkey": 3717, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6412.07, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-08", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " after the packa" }
+{ "l_orderkey": 3781, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21068.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "pendencies are b" }
+{ "l_orderkey": 4771, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19236.21, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-19", "l_commitdate": "1993-02-10", "l_receiptdate": "1993-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fluffily pendi" }
+{ "l_orderkey": 4806, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23816.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " bold pearls sublate blithely. quickly pe" }
+{ "l_orderkey": 4966, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 23816.26, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-11-29", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nt pearls haggle carefully slyly even " }
+{ "l_orderkey": 5184, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43052.47, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "se. carefully express pinto beans x" }
+{ "l_orderkey": 5189, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34808.38, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ideas. idle, final deposits de" }
+{ "l_orderkey": 5217, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21068.23, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-18", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ven ideas. requests amo" }
+{ "l_orderkey": 5286, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2748.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re fluffily" }
+{ "l_orderkey": 5415, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39388.43, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-09-14", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "yly blithely stealthy deposits. carefu" }
+{ "l_orderkey": 5442, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22900.25, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ake furiously. slyly express th" }
+{ "l_orderkey": 5510, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-16", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "n packages boost sly" }
+{ "l_orderkey": 5697, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 39388.43, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-08", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "blithely reg" }
+{ "l_orderkey": 5858, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 32976.36, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "osits wake quickly quickly sile" }
+{ "l_orderkey": 289, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40348.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic foxes. asymptotes " }
+{ "l_orderkey": 611, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35763.39, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nto beans " }
+{ "l_orderkey": 647, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-09-24", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r instructions. quickly unusu" }
+{ "l_orderkey": 774, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-13", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-03-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly even courts nag blith" }
+{ "l_orderkey": 931, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9170.1, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-01", "l_commitdate": "1993-01-09", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ajole quickly. slyly sil" }
+{ "l_orderkey": 967, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-08-07", "l_receiptdate": "1992-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "efully special ide" }
+{ "l_orderkey": 1411, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8253.09, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-08", "l_commitdate": "1995-03-04", "l_receiptdate": "1995-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "accounts. furiou" }
+{ "l_orderkey": 1825, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6419.07, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "fully ironic requests. requests cajole ex" }
+{ "l_orderkey": 1958, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 40348.44, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-11-30", "l_receiptdate": "1996-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c theodolites after the unusual deposit" }
+{ "l_orderkey": 2400, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21091.23, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-10-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ages lose carefully around the regula" }
+{ "l_orderkey": 2405, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 44933.49, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-03-23", "l_receiptdate": "1997-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cial requests. ironic, regu" }
+{ "l_orderkey": 2437, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress dolphins. furiously fin" }
+{ "l_orderkey": 2565, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22925.25, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ", express accounts. final id" }
+{ "l_orderkey": 2692, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests. bold, even foxes haggle slyl" }
+{ "l_orderkey": 2720, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 38514.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fter the inst" }
+{ "l_orderkey": 2944, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21091.23, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-03", "l_receiptdate": "1998-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " excuses? regular platelets e" }
+{ "l_orderkey": 3079, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36680.4, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-12-11", "l_receiptdate": "1997-10-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ide of the pending, special deposi" }
+{ "l_orderkey": 3621, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-08-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "al requests. fl" }
+{ "l_orderkey": 3872, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34846.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously against the ironic, unusual a" }
+{ "l_orderkey": 3943, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 29344.32, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual ideas into the furiously even pack" }
+{ "l_orderkey": 4005, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25676.28, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-01-24", "l_receiptdate": "1996-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly carefully ironic deposits. slyly" }
+{ "l_orderkey": 4262, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23842.26, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-29", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s boost slyly along the bold, iro" }
+{ "l_orderkey": 4355, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3668.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly blithely regular packag" }
+{ "l_orderkey": 4673, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7336.08, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely final re" }
+{ "l_orderkey": 4997, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4585.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-16", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cuses are furiously unusual asymptotes" }
+{ "l_orderkey": 5159, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42182.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s kindle slyly carefully regular" }
+{ "l_orderkey": 5189, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-12", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ial theodolites cajole slyly. slyly unus" }
+{ "l_orderkey": 5313, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31178.34, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccording to the blithely final account" }
+{ "l_orderkey": 5348, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14672.16, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-28", "l_commitdate": "1997-12-25", "l_receiptdate": "1998-03-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uriously thin pinto beans " }
+{ "l_orderkey": 5827, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 12838.14, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rges. fluffily pending " }
+{ "l_orderkey": 5924, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22008.24, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1995-12-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " use carefully. special, e" }
+{ "l_orderkey": 164, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts wake again" }
+{ "l_orderkey": 197, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22950.25, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-13", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s-- quickly final accounts" }
+{ "l_orderkey": 768, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1836.02, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-13", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ular courts. slyly dogged accou" }
+{ "l_orderkey": 901, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10098.11, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-13", "l_commitdate": "1998-10-19", "l_receiptdate": "1998-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ourts among the quickly expre" }
+{ "l_orderkey": 930, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ackages. fluffily e" }
+{ "l_orderkey": 965, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21114.23, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ld kindle carefully across th" }
+{ "l_orderkey": 1537, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he regular pack" }
+{ "l_orderkey": 1920, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5508.06, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-01", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-10-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l ideas boost slyly pl" }
+{ "l_orderkey": 1924, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-18", "l_receiptdate": "1996-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "silent requests cajole blithely final pack" }
+{ "l_orderkey": 1955, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1836.02, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ickly aroun" }
+{ "l_orderkey": 2150, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26622.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-02", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic theodolites. foxes ca" }
+{ "l_orderkey": 2151, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25704.28, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y special packages. carefully ironic instru" }
+{ "l_orderkey": 2246, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10098.11, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "quests alongside o" }
+{ "l_orderkey": 2404, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 37638.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-07-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " dolphins are" }
+{ "l_orderkey": 2658, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11934.13, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s kindle blithely regular accounts." }
+{ "l_orderkey": 2688, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41310.45, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-05-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sits run carefully" }
+{ "l_orderkey": 2818, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38556.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-19", "l_receiptdate": "1995-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar accounts wake carefully a" }
+{ "l_orderkey": 2854, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 11934.13, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " excuses wak" }
+{ "l_orderkey": 2880, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42228.46, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-06-05", "l_receiptdate": "1992-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eep quickly according to t" }
+{ "l_orderkey": 3015, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s above the fluffily final t" }
+{ "l_orderkey": 3040, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9180.1, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-16", "l_commitdate": "1993-06-24", "l_receiptdate": "1993-06-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely regular foxes haggle dari" }
+{ "l_orderkey": 3109, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ecial orbits are furiou" }
+{ "l_orderkey": 3175, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final requests x-r" }
+{ "l_orderkey": 3584, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 35802.39, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits. carefu" }
+{ "l_orderkey": 3777, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32130.35, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s. carefully express asymptotes accordi" }
+{ "l_orderkey": 4071, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ts cajole furiously along the" }
+{ "l_orderkey": 4263, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8262.09, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-04-29", "l_receiptdate": "1998-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "structions cajole quic" }
+{ "l_orderkey": 4833, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23868.26, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-13", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-05-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s packages. even gif" }
+{ "l_orderkey": 4960, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33048.36, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c, unusual accou" }
+{ "l_orderkey": 5186, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25704.28, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "al decoys. blit" }
+{ "l_orderkey": 5699, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 44064.48, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s. carefully regul" }
+{ "l_orderkey": 5829, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "after the furiously ironic ideas no" }
+{ "l_orderkey": 69, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 21137.23, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-10-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding accounts ca" }
+{ "l_orderkey": 164, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22056.24, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "side of the slyly unusual theodolites. f" }
+{ "l_orderkey": 230, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7352.08, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-03", "l_commitdate": "1994-01-20", "l_receiptdate": "1993-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g the instructions. fluffil" }
+{ "l_orderkey": 325, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32165.35, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-06", "l_commitdate": "1994-01-03", "l_receiptdate": "1993-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "packages wa" }
+{ "l_orderkey": 418, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28489.31, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "final theodolites. fluffil" }
+{ "l_orderkey": 481, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". quickly final accounts among the " }
+{ "l_orderkey": 999, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2757.03, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-22", "l_receiptdate": "1993-10-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nic, pending ideas. bl" }
+{ "l_orderkey": 1634, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19299.21, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-11-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y along the excuses." }
+{ "l_orderkey": 1636, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20218.22, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-08-18", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular, regu" }
+{ "l_orderkey": 1888, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8271.09, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-02-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages are blithely. carefu" }
+{ "l_orderkey": 2023, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22975.25, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-19", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake furiously among the slyly final" }
+{ "l_orderkey": 2149, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11028.12, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "riously bl" }
+{ "l_orderkey": 2304, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits cajole blithely e" }
+{ "l_orderkey": 2437, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9190.1, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-23", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts. even, ironic pl" }
+{ "l_orderkey": 2503, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-31", "l_receiptdate": "1993-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts haggle blithel" }
+{ "l_orderkey": 2694, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13785.15, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e blithely even platelets. special wa" }
+{ "l_orderkey": 2695, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40436.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-11-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. busy platelets boost" }
+{ "l_orderkey": 3044, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 43193.47, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-24", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly around the car" }
+{ "l_orderkey": 3207, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29408.32, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the slyly express foxes. bl" }
+{ "l_orderkey": 3425, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34003.37, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-10", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ngside of the furiously thin dol" }
+{ "l_orderkey": 3585, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36760.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-22", "l_commitdate": "1995-01-17", "l_receiptdate": "1995-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets affix. even asymptotes play care" }
+{ "l_orderkey": 3651, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tect quickly among the r" }
+{ "l_orderkey": 3719, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 16.0, "l_extendedprice": 14704.16, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " express asymptotes. ir" }
+{ "l_orderkey": 3874, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-06-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ideas throughout " }
+{ "l_orderkey": 4263, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uietly regular deposits. sly deposits w" }
+{ "l_orderkey": 4962, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42274.46, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " pinto beans grow about the sl" }
+{ "l_orderkey": 5127, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30327.33, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold deposits use carefully a" }
+{ "l_orderkey": 5344, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5514.06, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ithely about the pending plate" }
+{ "l_orderkey": 5538, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34922.38, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular pinto beans. silent ideas above " }
+{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+{ "l_orderkey": 326, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34960.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "es sleep slyly. carefully regular inst" }
+{ "l_orderkey": 865, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2760.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-14", "l_receiptdate": "1993-08-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fully regular the" }
+{ "l_orderkey": 1120, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-03", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "heodolites. quick re" }
+{ "l_orderkey": 1127, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26680.58, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. blithely r" }
+{ "l_orderkey": 1444, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 32200.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle furiou" }
+{ "l_orderkey": 1504, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-23", "l_receiptdate": "1992-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final packa" }
+{ "l_orderkey": 1509, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10120.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-04", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ily ironic packages nod carefully." }
+{ "l_orderkey": 1985, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1840.04, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-09", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " silent inst" }
+{ "l_orderkey": 2023, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20240.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ronic attainments. " }
+{ "l_orderkey": 2273, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-19", "l_commitdate": "1997-01-22", "l_receiptdate": "1997-02-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. furiou" }
+{ "l_orderkey": 2372, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4600.1, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-08", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ets against the " }
+{ "l_orderkey": 2625, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38640.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-10-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even accounts haggle furiously" }
+{ "l_orderkey": 2694, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11040.24, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "foxes atop the hockey pla" }
+{ "l_orderkey": 3202, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20240.44, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-16", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the express packages. fu" }
+{ "l_orderkey": 3330, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "haggle carefully alongside of the bold r" }
+{ "l_orderkey": 3778, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23920.52, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " against the fluffily" }
+{ "l_orderkey": 3910, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5520.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly sly platelets are fluffily slyly si" }
+{ "l_orderkey": 4193, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 46001.0, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-28", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " beans. regular accounts cajole. de" }
+{ "l_orderkey": 4386, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 16.0, "l_extendedprice": 14720.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously final pint" }
+{ "l_orderkey": 4773, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly pending theodolites cajole caref" }
+{ "l_orderkey": 5254, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 8280.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " wake blithely fluff" }
+{ "l_orderkey": 5348, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1997-12-20", "l_receiptdate": "1998-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "even foxes. epitap" }
+{ "l_orderkey": 5444, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 19320.42, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "furiously even theodolites." }
+{ "l_orderkey": 5476, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15640.34, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ng dependencies until the f" }
+{ "l_orderkey": 5510, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42320.92, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1993-02-09", "l_receiptdate": "1993-03-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "silent packages cajole doggedly regular " }
+{ "l_orderkey": 5956, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36800.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-11", "l_commitdate": "1998-07-19", "l_receiptdate": "1998-06-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final theodolites sleep carefully ironic c" }
+{ "l_orderkey": 39, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29472.64, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "heodolites sleep silently pending foxes. ac" }
+{ "l_orderkey": 67, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11052.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " even packages cajole" }
+{ "l_orderkey": 160, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31314.68, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "st sleep even gifts. dependencies along" }
+{ "l_orderkey": 481, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17499.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p blithely after t" }
+{ "l_orderkey": 549, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16578.36, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-09-11", "l_receiptdate": "1992-08-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ely regular accounts above the " }
+{ "l_orderkey": 1024, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 45129.98, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-10", "l_receiptdate": "1998-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully bold " }
+{ "l_orderkey": 1287, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23946.52, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-09-27", "l_receiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egular foxes. theodolites nag along t" }
+{ "l_orderkey": 1604, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21183.46, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "en requests. blithely fin" }
+{ "l_orderkey": 1667, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5526.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-11-16", "l_receiptdate": "1998-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "riously busy requests. blithely final a" }
+{ "l_orderkey": 1698, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20262.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "oward the furiously iro" }
+{ "l_orderkey": 1827, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6447.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "egular foxes" }
+{ "l_orderkey": 1921, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "to beans. even excuses integrate specia" }
+{ "l_orderkey": 1985, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 46051.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ate carefully. carefully" }
+{ "l_orderkey": 2468, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 39603.86, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-08-21", "l_receiptdate": "1997-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously eve" }
+{ "l_orderkey": 2530, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly ironic" }
+{ "l_orderkey": 2917, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18420.4, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-01-22", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly even ideas wa" }
+{ "l_orderkey": 2949, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3684.08, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "gular pinto beans wake alongside of the reg" }
+{ "l_orderkey": 3303, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13815.3, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-01-22", "l_receiptdate": "1998-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " detect sly" }
+{ "l_orderkey": 3331, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34998.76, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-24", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-08-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ymptotes haggle across the ca" }
+{ "l_orderkey": 3718, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36840.8, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "out the express deposits" }
+{ "l_orderkey": 3841, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28551.62, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "n theodolites shall promise carefully. qui" }
+{ "l_orderkey": 4064, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11052.24, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-08", "l_commitdate": "1996-12-18", "l_receiptdate": "1997-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ding to the requests" }
+{ "l_orderkey": 4327, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7368.16, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites cajole; unusual Tiresias" }
+{ "l_orderkey": 4581, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42366.92, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nag toward the carefully final accounts. " }
+{ "l_orderkey": 4642, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9210.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-04-28", "l_receiptdate": "1995-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "urts. even deposits nag beneath " }
+{ "l_orderkey": 5573, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29472.64, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egular depths haggl" }
+{ "l_orderkey": 67, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3688.08, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-04-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " cajole thinly expres" }
+{ "l_orderkey": 901, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33192.72, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-10-09", "l_receiptdate": "1998-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". accounts are care" }
+{ "l_orderkey": 966, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 38724.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sly ironic asymptotes hagg" }
+{ "l_orderkey": 1090, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4610.1, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1997-12-25", "l_receiptdate": "1998-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s above the " }
+{ "l_orderkey": 1285, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11064.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ss foxes. blithe theodolites cajole slyly" }
+{ "l_orderkey": 1348, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37802.82, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. platelets about the ca" }
+{ "l_orderkey": 1447, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7376.16, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1993-01-12", "l_receiptdate": "1992-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ost carefully " }
+{ "l_orderkey": 1510, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 46101.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-01", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "even packages. carefully regular fo" }
+{ "l_orderkey": 1637, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9220.2, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-03-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uriously? blithely even sauternes wake. " }
+{ "l_orderkey": 1667, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26738.58, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l accounts. furiously final courts h" }
+{ "l_orderkey": 2566, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8298.18, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-12-28", "l_receiptdate": "1992-12-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "blithely bold accounts? quickl" }
+{ "l_orderkey": 2626, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41490.9, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-11-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "deposits wake blithely according to " }
+{ "l_orderkey": 2757, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11064.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-01", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, eve" }
+{ "l_orderkey": 2913, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously pending realms. blithely even pac" }
+{ "l_orderkey": 3073, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eposits. fluffily" }
+{ "l_orderkey": 3362, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12908.28, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "even Tires" }
+{ "l_orderkey": 3492, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1995-01-18", "l_receiptdate": "1994-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ronic instructions u" }
+{ "l_orderkey": 3553, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16596.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": ". quickly ironic" }
+{ "l_orderkey": 3719, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32270.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly foxes. pending braids haggle furio" }
+{ "l_orderkey": 3970, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31348.68, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y final gifts are. carefully pe" }
+{ "l_orderkey": 3974, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-08", "l_receiptdate": "1996-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "dencies above the re" }
+{ "l_orderkey": 4418, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12908.28, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-18", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely regular requests. blith" }
+{ "l_orderkey": 4551, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-12", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ly ironic reques" }
+{ "l_orderkey": 4678, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21206.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-03", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ily sly deposi" }
+{ "l_orderkey": 4803, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-02-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "t blithely slyly special decoys. " }
+{ "l_orderkey": 4997, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42412.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ecial courts are carefully" }
+{ "l_orderkey": 5090, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-05-25", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ets integrate ironic, regul" }
+{ "l_orderkey": 5572, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests cajole. evenly ironic exc" }
+{ "l_orderkey": 37, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36920.8, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-21", "l_commitdate": "1992-08-01", "l_receiptdate": "1992-08-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "luffily regular requests. slyly final acco" }
+{ "l_orderkey": 512, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34151.74, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-20", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nic depths cajole? blithely b" }
+{ "l_orderkey": 708, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6461.14, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-16", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lly express ac" }
+{ "l_orderkey": 1025, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23075.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "xpress foxes. furiousl" }
+{ "l_orderkey": 1282, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12922.28, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ecial deposit" }
+{ "l_orderkey": 1382, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ter the carefully final excuses. blit" }
+{ "l_orderkey": 1412, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1846.04, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s among the requests are a" }
+{ "l_orderkey": 1767, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 46151.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-06-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y unusual foxe" }
+{ "l_orderkey": 1856, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 33228.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly even foxes kindle blithely even realm" }
+{ "l_orderkey": 2209, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36920.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ully special sheaves serve" }
+{ "l_orderkey": 2496, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35997.78, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-23", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "arefully special dependencies abo" }
+{ "l_orderkey": 2566, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16614.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1992-12-24", "l_receiptdate": "1992-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " braids according t" }
+{ "l_orderkey": 2598, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17537.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-09", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "nic packages. even accounts" }
+{ "l_orderkey": 2758, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15691.34, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-25", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts! qui" }
+{ "l_orderkey": 2759, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28613.62, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ithely aft" }
+{ "l_orderkey": 2786, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 22152.48, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-04", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ans. slyly unusual platelets detect. unus" }
+{ "l_orderkey": 2789, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37843.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "d packages-- fluffily specia" }
+{ "l_orderkey": 4609, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42458.92, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "r foxes. fluffily ironic ideas ha" }
+{ "l_orderkey": 4640, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 33228.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1996-03-09", "l_receiptdate": "1996-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "iously furious accounts boost. carefully" }
+{ "l_orderkey": 4834, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31382.68, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1996-11-26", "l_receiptdate": "1996-12-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ounts haggle bo" }
+{ "l_orderkey": 4896, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eposits hang carefully. sly" }
+{ "l_orderkey": 5347, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31382.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " pending deposits. fluffily regular senti" }
+{ "l_orderkey": 5987, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 923.02, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "refully final excuses haggle furiously ag" }
+{ "l_orderkey": 259, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 38808.84, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-11-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the slyly ironic pinto beans. fi" }
+{ "l_orderkey": 263, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20328.44, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-09-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "efully express fo" }
+{ "l_orderkey": 517, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21252.46, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " kindle. furiously bold requests mus" }
+{ "l_orderkey": 549, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35112.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-23", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits. carefully regular depos" }
+{ "l_orderkey": 551, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7392.16, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly slyly pending platel" }
+{ "l_orderkey": 677, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42504.92, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-02", "l_commitdate": "1994-02-12", "l_receiptdate": "1993-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ng theodolites. furiously unusual theodo" }
+{ "l_orderkey": 995, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16632.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-03", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even accounts unwind c" }
+{ "l_orderkey": 1572, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 37884.82, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-16", "l_commitdate": "1996-04-09", "l_receiptdate": "1996-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". pinto beans alongside" }
+{ "l_orderkey": 1733, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "slyly express deposits sleep abo" }
+{ "l_orderkey": 1761, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11088.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " sleep furiously. deposits are acco" }
+{ "l_orderkey": 1861, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21252.46, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "in packages sleep silent dolphins; sly" }
+{ "l_orderkey": 2083, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 34188.74, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ng the special foxes wake packages. f" }
+{ "l_orderkey": 2369, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27720.6, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pecial deposits sleep. blithely unusual w" }
+{ "l_orderkey": 2496, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27720.6, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake. ironic foxes cajole quickly. fu" }
+{ "l_orderkey": 2595, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17556.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-23", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ns are neve" }
+{ "l_orderkey": 2752, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36960.8, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "into beans are after the sly" }
+{ "l_orderkey": 2755, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10164.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular excuses sleep carefully." }
+{ "l_orderkey": 3008, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36960.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "esias. theodolites detect blithely " }
+{ "l_orderkey": 3010, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25872.56, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-03-28", "l_receiptdate": "1996-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ake carefully carefully even request" }
+{ "l_orderkey": 3079, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1848.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1997-11-17", "l_receiptdate": "1998-01-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly busy requests believ" }
+{ "l_orderkey": 3584, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5544.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "deposits across the" }
+{ "l_orderkey": 3623, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-01-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress ideas are furio" }
+{ "l_orderkey": 3845, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely bold ideas use. ex" }
+{ "l_orderkey": 4006, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13860.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-04-02", "l_receiptdate": "1995-02-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n deposits cajole slyl" }
+{ "l_orderkey": 4133, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32340.7, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-12-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "g above the quickly bold packages. ev" }
+{ "l_orderkey": 4166, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24024.52, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar dependencies. s" }
+{ "l_orderkey": 4192, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29568.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-23", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-07-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ounts are fluffily slyly bold req" }
+{ "l_orderkey": 4224, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3696.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-09-05", "l_receiptdate": "1997-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " even dinos. carefull" }
+{ "l_orderkey": 4260, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-06", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "al, pending accounts must" }
+{ "l_orderkey": 4261, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25872.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-08", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "packages. fluffily i" }
+{ "l_orderkey": 4481, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46201.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar packages. regula" }
+{ "l_orderkey": 5061, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24024.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-07", "l_commitdate": "1993-09-13", "l_receiptdate": "1993-11-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " cajole slyly. carefully spe" }
+{ "l_orderkey": 5126, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30492.66, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-04", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ipliers promise furiously whithout the " }
+{ "l_orderkey": 5510, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26796.58, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely fluffily ironic req" }
+{ "l_orderkey": 5572, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 31416.68, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-22", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-11-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "asymptotes integrate. s" }
+{ "l_orderkey": 1, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22200.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending foxes. slyly re" }
+{ "l_orderkey": 516, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10175.22, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ongside of the blithely final reque" }
+{ "l_orderkey": 768, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34225.74, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ending requests across the quickly" }
+{ "l_orderkey": 868, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 24975.54, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "oss the fluffily unusual pinto " }
+{ "l_orderkey": 1441, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 13875.3, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-21", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " dependencies-- cour" }
+{ "l_orderkey": 1508, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18500.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nic platelets. carefully final fra" }
+{ "l_orderkey": 1540, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5550.12, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-09-17", "l_receiptdate": "1992-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ing to the slyly express asymptote" }
+{ "l_orderkey": 1763, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14800.32, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ously pending asymptotes a" }
+{ "l_orderkey": 1767, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the bravely ironic requests i" }
+{ "l_orderkey": 1793, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38850.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-11-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uctions sleep carefully special, fl" }
+{ "l_orderkey": 1830, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8325.18, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-03-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st furiously among " }
+{ "l_orderkey": 2051, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ounts sleep fluffily even requ" }
+{ "l_orderkey": 2146, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 12950.28, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial, express a" }
+{ "l_orderkey": 2146, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 36075.78, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-11-06", "l_receiptdate": "1993-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uickly regular excuses detect. regular c" }
+{ "l_orderkey": 2561, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1997-12-28", "l_receiptdate": "1998-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "bold packages wake slyly. slyly" }
+{ "l_orderkey": 2688, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2775.06, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-04", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffily " }
+{ "l_orderkey": 2980, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45325.98, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-10-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hy packages sleep quic" }
+{ "l_orderkey": 3105, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 44400.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ending platelets wake carefully ironic inst" }
+{ "l_orderkey": 3265, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7400.16, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely ironic requests sleep slyly-- i" }
+{ "l_orderkey": 3523, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13875.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se slyly pending, sp" }
+{ "l_orderkey": 3585, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12025.26, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-01-22", "l_receiptdate": "1995-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ccording to the foxes. slyly iro" }
+{ "l_orderkey": 4709, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23125.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deposits grow. fluffily unusual accounts " }
+{ "l_orderkey": 5060, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 24975.54, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. ironic " }
+{ "l_orderkey": 5185, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ackages. slyly even requests" }
+{ "l_orderkey": 5505, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y alongside of the special requests." }
+{ "l_orderkey": 5762, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25900.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-22", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic foxes among the blithely qui" }
+{ "l_orderkey": 453, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 29632.64, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. fluffily bold packages cajole. unu" }
+{ "l_orderkey": 577, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23150.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ve slyly of the frets. careful" }
+{ "l_orderkey": 801, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10186.22, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y special pinto beans cajole " }
+{ "l_orderkey": 961, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35188.76, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-21", "l_commitdate": "1995-07-19", "l_receiptdate": "1995-08-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he blithely special requests. furiousl" }
+{ "l_orderkey": 1028, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 36114.78, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodoli" }
+{ "l_orderkey": 1089, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-24", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "g dolphins. deposits integrate. s" }
+{ "l_orderkey": 1442, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "c deposits haggle after the even" }
+{ "l_orderkey": 1541, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-08-07", "l_receiptdate": "1995-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y pending packages. blithely fi" }
+{ "l_orderkey": 1762, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13890.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old packages thrash. care" }
+{ "l_orderkey": 2146, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28706.62, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1992-10-24", "l_receiptdate": "1993-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lly even deposit" }
+{ "l_orderkey": 2567, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36114.78, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-10", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns. furiously final dependencies cajo" }
+{ "l_orderkey": 2758, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ake furious" }
+{ "l_orderkey": 2884, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "pending accounts about " }
+{ "l_orderkey": 3527, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 30558.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly alongside of " }
+{ "l_orderkey": 3968, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 41670.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-18", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully slyly fi" }
+{ "l_orderkey": 4007, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ter the accounts. expr" }
+{ "l_orderkey": 4131, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23150.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-01", "l_receiptdate": "1998-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uickly exp" }
+{ "l_orderkey": 4646, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20372.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cies are blithely after the slyly reg" }
+{ "l_orderkey": 4800, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-23", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nal accounts are blithely deposits. bol" }
+{ "l_orderkey": 4801, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31484.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-02-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final requests " }
+{ "l_orderkey": 5027, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34262.74, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-10-30", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ost slyly fluffily" }
+{ "l_orderkey": 5123, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12038.26, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "regular pearls" }
+{ "l_orderkey": 5572, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 22224.48, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " beans. foxes sleep fluffily across th" }
+{ "l_orderkey": 448, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32445.7, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ses nag quickly quickly ir" }
+{ "l_orderkey": 1124, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 39861.86, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-10-28", "l_receiptdate": "1998-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "across the " }
+{ "l_orderkey": 1411, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-03-02", "l_receiptdate": "1995-03-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d excuses. furiously final pear" }
+{ "l_orderkey": 1728, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31518.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "special req" }
+{ "l_orderkey": 1799, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 38934.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-04-28", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es pending " }
+{ "l_orderkey": 1826, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3708.08, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-12", "l_receiptdate": "1992-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "alongside of the quickly unusual re" }
+{ "l_orderkey": 1861, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "arefully unusual" }
+{ "l_orderkey": 2084, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25956.56, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cajole quickly carefu" }
+{ "l_orderkey": 2342, "l_partkey": 27, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20394.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-10", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. ironic " }
+{ "l_orderkey": 2405, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27810.6, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y final deposits are slyly caref" }
+{ "l_orderkey": 2534, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45423.98, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-20", "l_receiptdate": "1996-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sometimes regular requests. blithely unus" }
+{ "l_orderkey": 3782, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26883.58, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly unusual pinto beans. carefully fina" }
+{ "l_orderkey": 3783, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-02-17", "l_receiptdate": "1993-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing to the ideas. regular accounts de" }
+{ "l_orderkey": 4096, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-09-03", "l_receiptdate": "1992-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y final, even platelets. boldly" }
+{ "l_orderkey": 4129, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36153.78, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-10-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y regular foxes. slyly ironic deposits " }
+{ "l_orderkey": 4294, "l_partkey": 27, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14832.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lithely pint" }
+{ "l_orderkey": 4544, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7416.16, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "olites. fi" }
+{ "l_orderkey": 4640, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16686.36, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-06", "l_receiptdate": "1996-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "boost furiously accord" }
+{ "l_orderkey": 455, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40832.88, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " accounts sleep slyly ironic asymptote" }
+{ "l_orderkey": 485, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37120.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al escapades" }
+{ "l_orderkey": 645, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 8352.18, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-25", "l_commitdate": "1995-01-04", "l_receiptdate": "1995-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "special deposits. regular, final th" }
+{ "l_orderkey": 935, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21344.46, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular accounts about" }
+{ "l_orderkey": 1445, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15776.34, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ges. furiously regular pint" }
+{ "l_orderkey": 1506, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34336.74, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-01", "l_receiptdate": "1992-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully bold dolphins. accounts su" }
+{ "l_orderkey": 1509, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12992.28, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-09-25", "l_receiptdate": "1993-10-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nal realms" }
+{ "l_orderkey": 1985, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30624.66, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-01", "l_receiptdate": "1994-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are express packages. pendin" }
+{ "l_orderkey": 2240, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34336.74, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-16", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly after the packages? blithely si" }
+{ "l_orderkey": 2849, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-08", "l_receiptdate": "1996-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. carefully silent" }
+{ "l_orderkey": 2978, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6496.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-18", "l_commitdate": "1995-07-03", "l_receiptdate": "1995-07-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". final ideas are blithe" }
+{ "l_orderkey": 3495, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18560.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "posits are carefully; forges cajole qui" }
+{ "l_orderkey": 3520, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "deas should solve blithely among the ironi" }
+{ "l_orderkey": 3555, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 30624.66, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily regular a" }
+{ "l_orderkey": 3746, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10208.22, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic theodolites are among th" }
+{ "l_orderkey": 4034, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42688.92, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-22", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests. furiously unusual instructions wake" }
+{ "l_orderkey": 4039, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39904.86, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-01-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans believe bene" }
+{ "l_orderkey": 4161, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43616.94, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-10-29", "l_receiptdate": "1994-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r requests about the final, even foxes hag" }
+{ "l_orderkey": 4704, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6496.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ers wake car" }
+{ "l_orderkey": 5699, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 19488.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly final pla" }
+{ "l_orderkey": 5921, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24128.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hy dependenc" }
+{ "l_orderkey": 103, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-09-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic accou" }
+{ "l_orderkey": 132, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-08-27", "l_receiptdate": "1993-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "refully blithely bold acco" }
+{ "l_orderkey": 358, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16722.36, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olphins haggle ironic accounts. f" }
+{ "l_orderkey": 486, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2787.06, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ccounts ha" }
+{ "l_orderkey": 773, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26012.56, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1993-11-05", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he furiously slow deposits." }
+{ "l_orderkey": 838, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25083.54, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " pending pinto beans haggle about t" }
+{ "l_orderkey": 868, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12077.26, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-25", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "gged instructi" }
+{ "l_orderkey": 1028, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25083.54, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-03", "l_commitdate": "1994-02-07", "l_receiptdate": "1994-04-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ial accounts nag. slyly" }
+{ "l_orderkey": 1059, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6503.14, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furiously silent excuses are e" }
+{ "l_orderkey": 1575, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39018.84, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly pending pinto beans." }
+{ "l_orderkey": 1956, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10219.22, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-10-29", "l_receiptdate": "1993-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " the braids slee" }
+{ "l_orderkey": 2147, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46451.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "al accounts. even, even foxes wake" }
+{ "l_orderkey": 2630, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole. e" }
+{ "l_orderkey": 2658, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts cajole. pending packages affix" }
+{ "l_orderkey": 2784, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2787.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "n packages. foxes haggle quickly sile" }
+{ "l_orderkey": 2791, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24154.52, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1994-12-07", "l_receiptdate": "1995-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uriously special instructio" }
+{ "l_orderkey": 2978, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 30657.66, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. blithely unusual pack" }
+{ "l_orderkey": 3073, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13006.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ilently quiet epitaphs." }
+{ "l_orderkey": 3205, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lar accoun" }
+{ "l_orderkey": 3298, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23225.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-31", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly express f" }
+{ "l_orderkey": 3489, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-02", "l_commitdate": "1993-10-09", "l_receiptdate": "1993-08-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "xcuses? quickly stealthy dependenci" }
+{ "l_orderkey": 3558, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35302.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully permanently iron" }
+{ "l_orderkey": 3591, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19509.42, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "structions against " }
+{ "l_orderkey": 3778, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-22", "l_commitdate": "1993-08-18", "l_receiptdate": "1993-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tes affix carefully above the " }
+{ "l_orderkey": 4070, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nticing ideas. boldly" }
+{ "l_orderkey": 4161, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-11-17", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "he stealthily ironic foxes. ideas haggl" }
+{ "l_orderkey": 4263, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5574.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g the final, regular instructions: " }
+{ "l_orderkey": 4295, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45521.98, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "refully silent requests. f" }
+{ "l_orderkey": 4647, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26012.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " are above the fluffily fin" }
+{ "l_orderkey": 4738, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld, even packages. furio" }
+{ "l_orderkey": 4806, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7432.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-08", "l_commitdate": "1993-07-16", "l_receiptdate": "1993-05-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "requests boost blithely. qui" }
+{ "l_orderkey": 4997, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1858.04, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "counts. slyl" }
+{ "l_orderkey": 5254, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-16", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-09-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lyly regular accounts. furiously pendin" }
+{ "l_orderkey": 5410, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37160.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-17", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special accounts are along th" }
+{ "l_orderkey": 5569, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23225.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-07-18", "l_receiptdate": "1993-07-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " deposits cajole above" }
+{ "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1860.06, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
+{ "l_orderkey": 103, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29760.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-30", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-08-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages doze. special, regular deposit" }
+{ "l_orderkey": 646, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22320.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-20", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "regular accounts haggle dog" }
+{ "l_orderkey": 1121, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 43711.41, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly idle, i" }
+{ "l_orderkey": 1282, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9300.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-04-16", "l_receiptdate": "1992-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r theodolite" }
+{ "l_orderkey": 1702, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35341.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "as believe blithely. bo" }
+{ "l_orderkey": 1862, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38131.23, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully along" }
+{ "l_orderkey": 1925, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-04-06", "l_receiptdate": "1992-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "instructions sleep. pinto bea" }
+{ "l_orderkey": 2208, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 39991.29, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "es. accounts cajole. fi" }
+{ "l_orderkey": 2339, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26040.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-25", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e bold, even packag" }
+{ "l_orderkey": 2533, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-23", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-06-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ackages. blith" }
+{ "l_orderkey": 2662, "l_partkey": 30, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31621.02, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-10-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ding theodolites use carefully. p" }
+{ "l_orderkey": 3200, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10230.33, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-18", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "osits sleep fur" }
+{ "l_orderkey": 3655, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32551.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-11-16", "l_receiptdate": "1993-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "blithely even accounts! furiously regular" }
+{ "l_orderkey": 3714, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 40921.32, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-18", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. quickly ironic dugouts sublat" }
+{ "l_orderkey": 3781, "l_partkey": 30, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-23", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". theodolite" }
+{ "l_orderkey": 3970, "l_partkey": 30, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21390.69, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " above the final braids. regular" }
+{ "l_orderkey": 3973, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19530.63, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "equests. furiously" }
+{ "l_orderkey": 5025, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10230.33, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the carefully final esc" }
+{ "l_orderkey": 5377, "l_partkey": 30, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-05", "l_commitdate": "1997-05-25", "l_receiptdate": "1997-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dencies. carefully regular re" }
+{ "l_orderkey": 5891, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9300.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-02-07", "l_receiptdate": "1993-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nding requests. b" }
+{ "l_orderkey": 5986, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 930.03, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-21", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fix quickly quickly final deposits. fluffil" }
+{ "l_orderkey": 35, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26068.84, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly alongside of " }
+{ "l_orderkey": 354, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16758.54, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-31", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " about the carefully unusual " }
+{ "l_orderkey": 484, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45620.47, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ven accounts" }
+{ "l_orderkey": 994, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-06-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ainst the pending requests. packages sl" }
+{ "l_orderkey": 1028, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 22.0, "l_extendedprice": 20482.66, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-05-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " Tiresias alongside of the carefully spec" }
+{ "l_orderkey": 1185, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26068.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ke. slyly regular t" }
+{ "l_orderkey": 1476, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18620.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". bold deposits are carefully amo" }
+{ "l_orderkey": 1573, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-24", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully regular deposits. " }
+{ "l_orderkey": 1638, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "xcuses sleep furiou" }
+{ "l_orderkey": 1797, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-07-11", "l_receiptdate": "1996-08-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " cajole carefully. unusual Tiresias e" }
+{ "l_orderkey": 2403, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27930.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-08-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ackages sleep furiously pendin" }
+{ "l_orderkey": 2752, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38172.23, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tructions hag" }
+{ "l_orderkey": 2753, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6517.21, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "xpress ideas detect b" }
+{ "l_orderkey": 3335, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-25", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r packages cajole ac" }
+{ "l_orderkey": 3556, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40034.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1992-11-09", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "wake carefull" }
+{ "l_orderkey": 3781, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13965.45, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully blithe" }
+{ "l_orderkey": 3808, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 41896.35, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " carefully special" }
+{ "l_orderkey": 4197, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 44689.44, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final instructions. blithe, spe" }
+{ "l_orderkey": 4258, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42827.38, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously pend" }
+{ "l_orderkey": 4355, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 46551.5, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1997-01-01", "l_receiptdate": "1996-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " regular accounts boost along the " }
+{ "l_orderkey": 4608, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 33517.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages wake quickly slyly iron" }
+{ "l_orderkey": 4645, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 39103.26, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1994-10-22", "l_receiptdate": "1995-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e slyly regular pinto beans. thin" }
+{ "l_orderkey": 4705, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13034.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ain carefully amon" }
+{ "l_orderkey": 4864, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42827.38, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-24", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sts use carefully across the carefull" }
+{ "l_orderkey": 4870, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-11", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-10-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s haggle furiously. slyly ironic dinos" }
+{ "l_orderkey": 5249, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-29", "l_receiptdate": "1994-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole furiousl" }
+{ "l_orderkey": 5253, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32586.05, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-23", "l_commitdate": "1995-06-12", "l_receiptdate": "1995-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ven deposits. careful" }
+{ "l_orderkey": 5285, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34448.11, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-03-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uffily regu" }
+{ "l_orderkey": 5413, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 29792.96, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-23", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he quickly ironic ideas. slyly ironic ide" }
+{ "l_orderkey": 5415, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14896.48, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-10-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans haggle furiously" }
+{ "l_orderkey": 5575, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21413.69, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-09", "l_receiptdate": "1995-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "enticingly final requests. ironically" }
+{ "l_orderkey": 129, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uests. foxes cajole slyly after the ca" }
+{ "l_orderkey": 484, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 41941.35, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly final excuses boost slyly blithe" }
+{ "l_orderkey": 640, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 41941.35, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ong the qui" }
+{ "l_orderkey": 775, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14912.48, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "un quickly slyly" }
+{ "l_orderkey": 1028, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24232.78, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic platelets. carefully f" }
+{ "l_orderkey": 1447, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5592.18, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-10", "l_receiptdate": "1992-11-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as! regular packages poach above the" }
+{ "l_orderkey": 1475, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 30756.99, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1998-01-27", "l_receiptdate": "1998-01-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "quickly fluffy" }
+{ "l_orderkey": 1670, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38213.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely according to the sly" }
+{ "l_orderkey": 1762, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6524.21, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express packages wake slyly-- regul" }
+{ "l_orderkey": 2050, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10252.33, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ns. bold, final ideas cajole among the fi" }
+{ "l_orderkey": 2465, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7456.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s across the express deposits wak" }
+{ "l_orderkey": 2723, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-27", "l_commitdate": "1995-11-29", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al, special r" }
+{ "l_orderkey": 2817, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4660.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-07", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "furiously unusual theodolites use furiou" }
+{ "l_orderkey": 2885, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13980.45, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-31", "l_commitdate": "1992-11-24", "l_receiptdate": "1992-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "odolites. boldly pending packages han" }
+{ "l_orderkey": 3553, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 37281.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " slyly pending asymptotes against the furi" }
+{ "l_orderkey": 3716, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. quickly sly ideas slee" }
+{ "l_orderkey": 4038, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the furiously regu" }
+{ "l_orderkey": 4197, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26096.84, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular pin" }
+{ "l_orderkey": 4419, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. furious" }
+{ "l_orderkey": 4449, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. blithely final " }
+{ "l_orderkey": 4580, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-12-30", "l_receiptdate": "1994-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular, pending deposits. fina" }
+{ "l_orderkey": 4770, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38213.23, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-04", "l_commitdate": "1995-08-08", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ithely even packages sleep caref" }
+{ "l_orderkey": 4900, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18640.6, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-09-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yers. accounts affix somet" }
+{ "l_orderkey": 4933, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 44737.44, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-10", "l_commitdate": "1995-10-03", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ideas. sly" }
+{ "l_orderkey": 5060, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26096.84, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-08-11", "l_receiptdate": "1992-10-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c requests" }
+{ "l_orderkey": 5127, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18640.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-02-26", "l_receiptdate": "1997-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "dolites about the final platelets w" }
+{ "l_orderkey": 5249, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12116.39, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ites. finally exp" }
+{ "l_orderkey": 5603, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 45669.47, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic, pending dependencies print" }
+{ "l_orderkey": 388, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-02-26", "l_receiptdate": "1993-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "accounts sleep furiously" }
+{ "l_orderkey": 451, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "express excuses. blithely ironic pin" }
+{ "l_orderkey": 483, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7464.24, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-22", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "osits. carefully fin" }
+{ "l_orderkey": 512, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11196.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-21", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "old furiously express deposits. specia" }
+{ "l_orderkey": 579, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36388.17, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ncies. furiously final r" }
+{ "l_orderkey": 1156, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1997-01-14", "l_receiptdate": "1996-12-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "dolphins. fluffily ironic packages sleep re" }
+{ "l_orderkey": 1280, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12129.39, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "blithely final accounts use evenly " }
+{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2799.09, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly enticing requ" }
+{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17727.57, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-20", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-07-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ught to are bold foxes" }
+{ "l_orderkey": 1444, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1995-02-18", "l_receiptdate": "1994-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ss requests. ironic ideas wake above" }
+{ "l_orderkey": 2053, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31723.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ions. unusual dependencies" }
+{ "l_orderkey": 2215, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27990.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-09-10", "l_receiptdate": "1996-08-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ckages caj" }
+{ "l_orderkey": 2784, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41986.35, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly along the asymptotes. reque" }
+{ "l_orderkey": 2787, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3732.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-26", "l_commitdate": "1995-11-26", "l_receiptdate": "1996-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. instructions nag furiously according " }
+{ "l_orderkey": 2855, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46651.5, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans. deposits " }
+{ "l_orderkey": 3526, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18660.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "kages. bold, special requests detect sl" }
+{ "l_orderkey": 3555, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27057.87, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. carefully s" }
+{ "l_orderkey": 3747, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely bold orbits mold furiously blit" }
+{ "l_orderkey": 4163, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12129.39, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-17", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "phins wake. pending requests inte" }
+{ "l_orderkey": 4227, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7464.24, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages since the bold, u" }
+{ "l_orderkey": 4359, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20526.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-28", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts wake ironic deposits. ironic" }
+{ "l_orderkey": 5346, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5598.18, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "escapades sleep furiously beside the " }
+{ "l_orderkey": 5351, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43852.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-08-08", "l_receiptdate": "1998-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. grouches cajole. sile" }
+{ "l_orderkey": 5574, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully final dugouts. express foxes nag " }
+{ "l_orderkey": 5733, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36388.17, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "side of the" }
+{ "l_orderkey": 33, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 38295.23, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1994-01-24", "l_receiptdate": "1993-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unusual packages doubt caref" }
+{ "l_orderkey": 165, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2802.09, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "riously requests. depos" }
+{ "l_orderkey": 230, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7472.24, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1994-01-05", "l_receiptdate": "1993-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal ideas. silent, reg" }
+{ "l_orderkey": 322, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2802.09, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-07-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending, ironic deposits along the blith" }
+{ "l_orderkey": 576, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5604.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. slyly even sauternes a" }
+{ "l_orderkey": 645, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-03-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep. slyly even " }
+{ "l_orderkey": 961, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27086.87, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l accounts use blithely against the" }
+{ "l_orderkey": 1284, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8406.27, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "after the pending" }
+{ "l_orderkey": 1381, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously regular package" }
+{ "l_orderkey": 1443, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43899.41, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-05", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "carefully ironic requests sl" }
+{ "l_orderkey": 1478, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19614.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " fluffily pending acc" }
+{ "l_orderkey": 1571, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 22416.72, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "warthogs wake carefully acro" }
+{ "l_orderkey": 1733, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gainst the final deposits. carefully final " }
+{ "l_orderkey": 1766, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites above the final, regular acc" }
+{ "l_orderkey": 1920, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-22", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ickly ironic d" }
+{ "l_orderkey": 1924, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28954.93, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " the slyly regular foxes. ruthle" }
+{ "l_orderkey": 2020, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46701.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-08-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ts against the pending ideas serve along" }
+{ "l_orderkey": 2272, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37361.2, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely ir" }
+{ "l_orderkey": 2275, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28020.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re slyly slyly special idea" }
+{ "l_orderkey": 2307, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ven instructions wake fluffily " }
+{ "l_orderkey": 2497, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26152.84, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-21", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ouches. special, regular requests" }
+{ "l_orderkey": 2722, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14944.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts besides the fluffy," }
+{ "l_orderkey": 3270, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "promise carefully." }
+{ "l_orderkey": 3845, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41097.32, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s haggle among the fluffily regula" }
+{ "l_orderkey": 4614, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-09-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ickly furio" }
+{ "l_orderkey": 4646, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans sleep car" }
+{ "l_orderkey": 4675, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24284.78, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-12-29", "l_receiptdate": "1993-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nts. express requests are quickly " }
+{ "l_orderkey": 4707, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial sheaves boost blithely accor" }
+{ "l_orderkey": 4743, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25218.81, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-26", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aids use. express deposits" }
+{ "l_orderkey": 4899, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1994-01-10", "l_receiptdate": "1993-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " foxes eat" }
+{ "l_orderkey": 5089, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 35493.14, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-09-11", "l_receiptdate": "1992-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular instructions are" }
+{ "l_orderkey": 5285, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22416.72, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-19", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ess packages. quick, even deposits snooze b" }
+{ "l_orderkey": 5345, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "leep slyly regular fox" }
+{ "l_orderkey": 68, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43011.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "egular dependencies affix ironically along " }
+{ "l_orderkey": 71, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42076.35, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-23", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " ironic packages believe blithely a" }
+{ "l_orderkey": 326, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-10-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cies sleep quick" }
+{ "l_orderkey": 418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2805.09, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-29", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly furiously regular w" }
+{ "l_orderkey": 1697, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17765.57, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1996-11-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ons? special, special accounts after" }
+{ "l_orderkey": 1701, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24310.78, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " accounts. blithely pending pinto be" }
+{ "l_orderkey": 2048, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6545.21, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lent platelets boost deposits. carefully sp" }
+{ "l_orderkey": 2049, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-25", "l_receiptdate": "1995-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages are slyly alongside" }
+{ "l_orderkey": 2371, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19635.63, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-02-14", "l_receiptdate": "1998-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gle furiously regu" }
+{ "l_orderkey": 2724, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 935.03, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1994-11-27", "l_receiptdate": "1995-01-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly carefully blithe theodolites-- pl" }
+{ "l_orderkey": 2880, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-26", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "even requests. quick" }
+{ "l_orderkey": 2914, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-06-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s integrate. bold deposits sleep req" }
+{ "l_orderkey": 3270, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10285.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " solve at the regular deposits. " }
+{ "l_orderkey": 3460, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "o the even deposits" }
+{ "l_orderkey": 4230, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28050.9, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s. final excuses across the" }
+{ "l_orderkey": 4257, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4675.15, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-06-05", "l_receiptdate": "1995-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "n deposits. furiously e" }
+{ "l_orderkey": 4258, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20570.66, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e regular, even asym" }
+{ "l_orderkey": 4418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29920.96, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-02", "l_receiptdate": "1993-05-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. bold pinto b" }
+{ "l_orderkey": 4611, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " final pinto beans. permanent, sp" }
+{ "l_orderkey": 4769, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14960.48, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " deposits. slyly even asymptote" }
+{ "l_orderkey": 4804, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38336.23, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". deposits haggle express tithes?" }
+{ "l_orderkey": 4833, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-16", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y pending packages sleep blithely regular r" }
+{ "l_orderkey": 5153, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39271.26, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re thinly. ironic" }
+{ "l_orderkey": 5315, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11220.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts. furiously ironi" }
+{ "l_orderkey": 5414, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21505.69, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-08-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e bold, express dolphins. spec" }
+{ "l_orderkey": 5856, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 32726.05, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1994-12-23", "l_receiptdate": "1994-11-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "excuses. finally ir" }
+{ "l_orderkey": 134, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11232.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nts are quic" }
+{ "l_orderkey": 258, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23400.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep pending packages." }
+{ "l_orderkey": 962, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25272.81, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-11", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y slyly express deposits. final i" }
+{ "l_orderkey": 1126, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41185.32, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-05-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. carefully special" }
+{ "l_orderkey": 1154, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16848.54, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-26", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular excuses cajole blithely. fi" }
+{ "l_orderkey": 1575, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36505.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ironic requests snooze ironic, regular acc" }
+{ "l_orderkey": 1924, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 15912.51, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-31", "l_commitdate": "1996-11-12", "l_receiptdate": "1997-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e carefully theodolites. ironically ironic " }
+{ "l_orderkey": 1988, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25272.81, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "uests. regular requests are according to t" }
+{ "l_orderkey": 2342, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-31", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ffily. unusual pinto beans wake c" }
+{ "l_orderkey": 2371, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 29952.96, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-02-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ruthless accounts. " }
+{ "l_orderkey": 2404, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "from the final orbits? even pinto beans hag" }
+{ "l_orderkey": 2853, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully slyly quick packages. final c" }
+{ "l_orderkey": 3296, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1994-12-23", "l_receiptdate": "1995-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "carefully fur" }
+{ "l_orderkey": 3302, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42121.35, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "counts use quickl" }
+{ "l_orderkey": 3303, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24336.78, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly permanent requests w" }
+{ "l_orderkey": 3395, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35569.14, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " silent accounts are blithely" }
+{ "l_orderkey": 3521, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26208.84, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly above the slyly final" }
+{ "l_orderkey": 4038, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-09", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " special instructions. packa" }
+{ "l_orderkey": 4131, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7488.24, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-03", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " after the furiously ironic d" }
+{ "l_orderkey": 4484, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27144.87, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " wake blithely ironic" }
+{ "l_orderkey": 4641, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14040.45, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-25", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s. carefully even exc" }
+{ "l_orderkey": 4768, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4680.15, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "egular accounts. bravely final fra" }
+{ "l_orderkey": 4833, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17784.57, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-07-09", "l_receiptdate": "1996-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y quick theodolit" }
+{ "l_orderkey": 4901, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 38377.23, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-18", "l_commitdate": "1998-02-18", "l_receiptdate": "1998-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully bold packages affix carefully eve" }
+{ "l_orderkey": 5666, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13104.42, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar deposits nag against the slyly final d" }
+{ "l_orderkey": 163, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25299.81, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ously express dependen" }
+{ "l_orderkey": 576, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5622.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. ironic multipliers " }
+{ "l_orderkey": 678, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20614.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-10", "l_commitdate": "1993-04-29", "l_receiptdate": "1993-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "de of the carefully even requests. bl" }
+{ "l_orderkey": 1026, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5622.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-07", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "to beans. special, regular packages hagg" }
+{ "l_orderkey": 1088, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10307.33, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-30", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "inal requests. fluffily express theod" }
+{ "l_orderkey": 1220, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2811.09, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final theodolites. blithely silent " }
+{ "l_orderkey": 1319, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11244.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "packages integrate furiously. expres" }
+{ "l_orderkey": 1923, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8433.27, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-29", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lites. ironic instructions integrate bravel" }
+{ "l_orderkey": 1923, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 46851.5, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-04", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-11-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uickly along the bold courts. bold the" }
+{ "l_orderkey": 2500, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31859.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " stealthy a" }
+{ "l_orderkey": 2980, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1874.06, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "enly across the special, pending packag" }
+{ "l_orderkey": 2981, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13118.42, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-30", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "kages detect furiously express requests." }
+{ "l_orderkey": 3074, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46851.5, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending requests haggle s" }
+{ "l_orderkey": 3589, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39355.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-11", "l_commitdate": "1994-07-17", "l_receiptdate": "1994-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he blithely unusual pac" }
+{ "l_orderkey": 4224, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18740.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-09", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts promise across the requests. blith" }
+{ "l_orderkey": 4645, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 39355.26, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-02", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "regular pinto beans amon" }
+{ "l_orderkey": 5057, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35607.14, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-24", "l_commitdate": "1997-09-07", "l_receiptdate": "1997-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "packages. stealthily bold wa" }
+{ "l_orderkey": 5, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-10-13", "l_receiptdate": "1994-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites. fluffily unusual" }
+{ "l_orderkey": 69, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2814.09, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-07-27", "l_receiptdate": "1994-06-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithely final d" }
+{ "l_orderkey": 70, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34707.11, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-03-16", "l_receiptdate": "1994-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "n accounts are. q" }
+{ "l_orderkey": 226, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32831.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "osits cajole. final, even foxes a" }
+{ "l_orderkey": 322, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4690.15, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-15", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special grouches sleep quickly instructio" }
+{ "l_orderkey": 804, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19698.63, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-12", "l_commitdate": "1993-06-06", "l_receiptdate": "1993-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ular, ironic foxes. quickly even accounts" }
+{ "l_orderkey": 992, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s use silently. blithely regular ideas b" }
+{ "l_orderkey": 1026, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33769.08, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-14", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st the ide" }
+{ "l_orderkey": 1091, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 37521.2, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets. regular packag" }
+{ "l_orderkey": 1414, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36583.17, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-22", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "quickly aro" }
+{ "l_orderkey": 1699, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "to the final requests are carefully silent " }
+{ "l_orderkey": 2023, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1876.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-27", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing packages. fluffily silen" }
+{ "l_orderkey": 2213, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40335.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages are along the carefully bol" }
+{ "l_orderkey": 3014, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28140.9, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " final foxes." }
+{ "l_orderkey": 3104, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24388.78, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1993-12-05", "l_receiptdate": "1994-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "es boost carefully. slyly " }
+{ "l_orderkey": 3266, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40335.29, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-04", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular asymptotes use careful" }
+{ "l_orderkey": 3269, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43149.38, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "final asymptotes nag" }
+{ "l_orderkey": 3270, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 41273.32, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " accounts. carefully even " }
+{ "l_orderkey": 3462, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13132.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "yly. blithely bold theodolites wa" }
+{ "l_orderkey": 3904, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20636.66, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-02-09", "l_receiptdate": "1998-02-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "structions cajole carefully. carefully f" }
+{ "l_orderkey": 4033, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "t the blithely dogg" }
+{ "l_orderkey": 4613, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15946.51, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "liers cajole a" }
+{ "l_orderkey": 4864, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35645.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-07", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ording to the ironic, ir" }
+{ "l_orderkey": 4993, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-21", "l_commitdate": "1994-10-31", "l_receiptdate": "1994-09-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ular, pending packages at the even packa" }
+{ "l_orderkey": 5601, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27202.87, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-02-24", "l_receiptdate": "1992-04-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " ironic ideas. final" }
+{ "l_orderkey": 5798, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8442.27, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e blithely" }
+{ "l_orderkey": 100, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13146.42, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y. furiously ironic ideas gr" }
+{ "l_orderkey": 482, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 43195.38, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-06-05", "l_receiptdate": "1996-08-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "furiously thin realms. final, fina" }
+{ "l_orderkey": 612, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26292.84, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-05", "l_receiptdate": "1992-12-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly regular asym" }
+{ "l_orderkey": 896, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44134.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even pinto beans integrate. b" }
+{ "l_orderkey": 1447, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8451.27, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-15", "l_commitdate": "1993-01-07", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "counts wake s" }
+{ "l_orderkey": 1569, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15024.48, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-26", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "deposits. blithely final asymptotes ac" }
+{ "l_orderkey": 1600, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7512.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-04-22", "l_receiptdate": "1993-03-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cajole furiously fluf" }
+{ "l_orderkey": 1603, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 939.03, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-17", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "d accounts. special warthogs use fur" }
+{ "l_orderkey": 1958, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 29.0, "l_extendedprice": 27231.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-11-06", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "final requests nag according to the " }
+{ "l_orderkey": 2435, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7512.24, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e fluffily quickly final accounts. care" }
+{ "l_orderkey": 2535, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11268.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uses sleep among the packages. excuses " }
+{ "l_orderkey": 2596, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17841.57, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-02", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-09-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ias mold! sp" }
+{ "l_orderkey": 3396, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 16902.54, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-08-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "l requests haggle furiously along the fur" }
+{ "l_orderkey": 3461, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41317.32, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " haggle quickly even ideas. fin" }
+{ "l_orderkey": 3588, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 43195.38, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-06-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " slyly ironic deposits sublate ab" }
+{ "l_orderkey": 4515, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14085.45, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-26", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-06-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "posits wake" }
+{ "l_orderkey": 4774, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3756.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "xes according to the foxes wake above the f" }
+{ "l_orderkey": 4994, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 37561.2, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits. regula" }
+{ "l_orderkey": 5287, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30048.96, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-01-27", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "heodolites haggle caref" }
+{ "l_orderkey": 5344, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19719.63, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-09-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "xes. furiously even pinto beans sleep f" }
+{ "l_orderkey": 5570, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14085.45, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "beans nag slyly special, regular pack" }
+{ "l_orderkey": 5958, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16902.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "regular requests haggle" }
+{ "l_orderkey": 129, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31021.32, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts nag bravely. fluffily" }
+{ "l_orderkey": 289, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45121.92, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sits cajole. bold pinto beans x-ray fl" }
+{ "l_orderkey": 417, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36661.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y regular requests wake along " }
+{ "l_orderkey": 773, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 40421.72, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-11-20", "l_receiptdate": "1993-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "furiously bold dependencies. blithel" }
+{ "l_orderkey": 931, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16920.72, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly ironic re" }
+{ "l_orderkey": 993, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9400.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "encies wake fur" }
+{ "l_orderkey": 1252, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-10-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake carefully-- packages sleep. quick " }
+{ "l_orderkey": 1507, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 31021.32, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-23", "l_receiptdate": "1993-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " asymptotes nag furiously above t" }
+{ "l_orderkey": 1543, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8460.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ravely special requests " }
+{ "l_orderkey": 1667, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-11-24", "l_receiptdate": "1998-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "around the pinto beans. express, special" }
+{ "l_orderkey": 1924, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19740.84, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-21", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " blithely reg" }
+{ "l_orderkey": 1926, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27261.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hily unusual packages are fluffily am" }
+{ "l_orderkey": 2818, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 30081.28, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "arefully! ac" }
+{ "l_orderkey": 3110, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15040.64, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-10", "l_commitdate": "1995-02-06", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the regular acco" }
+{ "l_orderkey": 3139, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 43241.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-03-04", "l_receiptdate": "1992-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "of the unusual, unusual re" }
+{ "l_orderkey": 3170, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11280.48, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-17", "l_receiptdate": "1998-02-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing accounts along the speci" }
+{ "l_orderkey": 3366, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-25", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully about " }
+{ "l_orderkey": 3462, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40421.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-01", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully. final, final ideas sleep slyly" }
+{ "l_orderkey": 3906, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47002.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-09-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ke slyly. stealt" }
+{ "l_orderkey": 3973, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37601.6, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-03", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g the carefully blithe f" }
+{ "l_orderkey": 4002, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously furiously special theodoli" }
+{ "l_orderkey": 4064, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14100.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "braids affix across the regular sheave" }
+{ "l_orderkey": 4231, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32901.4, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "le quickly regular, unus" }
+{ "l_orderkey": 4292, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 940.04, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-07", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " the furiously ev" }
+{ "l_orderkey": 4646, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 35721.52, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al platelets cajole. slyly final dol" }
+{ "l_orderkey": 4802, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual accounts wake blithely. b" }
+{ "l_orderkey": 4935, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34781.48, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y even dependencies nag a" }
+{ "l_orderkey": 4995, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8460.36, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ironic packages cajole across t" }
+{ "l_orderkey": 4997, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4700.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "aggle slyly alongside of the slyly i" }
+{ "l_orderkey": 5286, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y special a" }
+{ "l_orderkey": 5347, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-04-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ldly pending asymptotes ki" }
+{ "l_orderkey": 5505, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16920.72, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-25", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " to the quickly express pac" }
+{ "l_orderkey": 5829, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ithely; accounts cajole ideas. regular foxe" }
+{ "l_orderkey": 5959, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34781.48, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-06-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "endencies. brai" }
+{ "l_orderkey": 67, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21643.92, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly regular deposit" }
+{ "l_orderkey": 98, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26349.12, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-25", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " pending, regular accounts s" }
+{ "l_orderkey": 226, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully pending pi" }
+{ "l_orderkey": 517, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8469.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-05-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " slyly stealthily express instructions. " }
+{ "l_orderkey": 1314, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10351.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tegrate furious" }
+{ "l_orderkey": 1316, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dugouts. co" }
+{ "l_orderkey": 2085, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-01-11", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". carefully e" }
+{ "l_orderkey": 2279, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35759.52, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s above the furiously express dep" }
+{ "l_orderkey": 2406, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "gular accounts caj" }
+{ "l_orderkey": 2852, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the blithe" }
+{ "l_orderkey": 2917, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34818.48, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-12", "l_commitdate": "1998-02-03", "l_receiptdate": "1997-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dependencies. express " }
+{ "l_orderkey": 3073, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23526.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nag asymptotes. pinto beans sleep " }
+{ "l_orderkey": 3170, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31995.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s about the fluffily final de" }
+{ "l_orderkey": 3367, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25408.08, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-03-16", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kly even instructions caj" }
+{ "l_orderkey": 3459, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-09-09", "l_receiptdate": "1994-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ntly speci" }
+{ "l_orderkey": 3617, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20702.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily even accounts. packages sleep blithe" }
+{ "l_orderkey": 3878, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18820.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the furiously careful ideas cajole slyly sl" }
+{ "l_orderkey": 3941, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44228.88, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully pending" }
+{ "l_orderkey": 4001, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17879.76, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. carefully ironi" }
+{ "l_orderkey": 4165, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11292.48, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-20", "l_receiptdate": "1997-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nwind slow theodolites. carefully pending " }
+{ "l_orderkey": 4448, "l_partkey": 41, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32936.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-07-27", "l_receiptdate": "1998-10-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "aggle carefully alongside of the q" }
+{ "l_orderkey": 4512, "l_partkey": 41, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-16", "l_receiptdate": "1995-12-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly regular pinto beans. carefully bold depo" }
+{ "l_orderkey": 4869, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29172.24, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-02-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ins. always unusual ideas across the ir" }
+{ "l_orderkey": 4896, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 17879.76, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-13", "l_commitdate": "1992-11-13", "l_receiptdate": "1993-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nusual requ" }
+{ "l_orderkey": 4964, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 39523.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-08-28", "l_receiptdate": "1997-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " hinder. idly even" }
+{ "l_orderkey": 197, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-08", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "use slyly slyly silent depo" }
+{ "l_orderkey": 260, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25435.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-23", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-04-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ions according to the" }
+{ "l_orderkey": 327, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " asymptotes are fu" }
+{ "l_orderkey": 771, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6594.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-18", "l_commitdate": "1995-08-31", "l_receiptdate": "1995-06-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "theodolites after the fluffily express " }
+{ "l_orderkey": 903, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-22", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-11-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y final platelets sublate among the " }
+{ "l_orderkey": 1152, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5652.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-11-05", "l_receiptdate": "1994-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p furiously; packages above th" }
+{ "l_orderkey": 1216, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16956.72, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-01-28", "l_receiptdate": "1993-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final packages nod " }
+{ "l_orderkey": 1218, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "press furio" }
+{ "l_orderkey": 1413, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5652.24, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lithely excuses. f" }
+{ "l_orderkey": 1571, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9420.4, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1993-02-13", "l_receiptdate": "1992-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lets. carefully regular ideas wake" }
+{ "l_orderkey": 1604, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14130.6, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-09-03", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions haggle" }
+{ "l_orderkey": 1702, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-31", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ackages sleep. furiously even excuses snooz" }
+{ "l_orderkey": 1767, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ing to the slyly fin" }
+{ "l_orderkey": 2310, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45217.92, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-20", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep slyly alongside of the " }
+{ "l_orderkey": 2342, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11304.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "print blithely even deposits. carefull" }
+{ "l_orderkey": 2560, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits affix quickly. unusual, eve" }
+{ "l_orderkey": 2566, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2826.12, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-30", "l_receiptdate": "1992-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ckages are ironic Tiresias. furious" }
+{ "l_orderkey": 2599, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24493.04, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-21", "l_receiptdate": "1996-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nag carefully " }
+{ "l_orderkey": 2659, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "idle tithes" }
+{ "l_orderkey": 2882, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28261.2, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-10-13", "l_receiptdate": "1995-10-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "among the furiously even theodolites. regu" }
+{ "l_orderkey": 2944, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 41449.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ickly. regular requests haggle. idea" }
+{ "l_orderkey": 3268, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37681.6, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly. bold, eve" }
+{ "l_orderkey": 3488, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11304.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e slyly; furiously final packages wak" }
+{ "l_orderkey": 3585, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 42391.8, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-02-19", "l_receiptdate": "1995-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "are blithely c" }
+{ "l_orderkey": 4576, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "detect slyly." }
+{ "l_orderkey": 4579, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly across the " }
+{ "l_orderkey": 4645, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 25435.08, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously express pinto beans. ironic depos" }
+{ "l_orderkey": 4837, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ing requests are blithely regular instructi" }
+{ "l_orderkey": 4994, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22608.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-08-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s. slyly ironic deposits cajole f" }
+{ "l_orderkey": 5442, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages. accounts haggle dependencies. f" }
+{ "l_orderkey": 5765, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19782.84, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-05", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ole furiously. quick, special dependencies " }
+{ "l_orderkey": 326, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44322.88, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-07-04", "l_receiptdate": "1995-10-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " special accounts sleep " }
+{ "l_orderkey": 678, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10373.44, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-28", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess deposits dazzle f" }
+{ "l_orderkey": 707, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20746.88, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " kindle ironically" }
+{ "l_orderkey": 901, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 34892.48, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-01", "l_commitdate": "1998-09-13", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ickly final deposits " }
+{ "l_orderkey": 1127, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "l instructions boost blithely according " }
+{ "l_orderkey": 1639, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35835.52, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-23", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y regular packages. b" }
+{ "l_orderkey": 1892, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-05-09", "l_receiptdate": "1994-05-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "hes nod furiously around the instruc" }
+{ "l_orderkey": 2023, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27348.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual instructions. bli" }
+{ "l_orderkey": 2052, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15088.64, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-30", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final deposits cajole according " }
+{ "l_orderkey": 2208, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "al foxes will hav" }
+{ "l_orderkey": 2371, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31120.32, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deas are. express r" }
+{ "l_orderkey": 2372, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39607.68, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-02", "l_receiptdate": "1998-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar packages. regular" }
+{ "l_orderkey": 2659, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19803.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-23", "l_commitdate": "1994-02-10", "l_receiptdate": "1994-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y beyond the furiously even co" }
+{ "l_orderkey": 2978, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24519.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "as haggle against the carefully express dep" }
+{ "l_orderkey": 3333, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42436.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-04", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "dolites. quickly r" }
+{ "l_orderkey": 3395, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 40550.72, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckages above the furiously regu" }
+{ "l_orderkey": 3555, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23576.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sual packages. quickly " }
+{ "l_orderkey": 3653, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8487.36, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-08-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tes: blithely bo" }
+{ "l_orderkey": 3808, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26405.12, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lly final accounts alo" }
+{ "l_orderkey": 3811, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17917.76, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-07-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s boost blithely furiou" }
+{ "l_orderkey": 4069, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30177.28, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unts. deposit" }
+{ "l_orderkey": 4259, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13202.56, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1997-11-21", "l_receiptdate": "1998-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously pending excuses. ideas hagg" }
+{ "l_orderkey": 4324, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 29234.24, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-23", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "efully flu" }
+{ "l_orderkey": 4517, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-08", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "refully pending acco" }
+{ "l_orderkey": 5191, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25462.08, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tructions nag bravely within the re" }
+{ "l_orderkey": 5281, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31120.32, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly brave foxes. bold deposits above the " }
+{ "l_orderkey": 5285, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11316.48, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-22", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-05-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " deposits-- quickly bold requests hag" }
+{ "l_orderkey": 5444, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37721.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-09", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ously bold ideas. instructions wake slyl" }
+{ "l_orderkey": 5793, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7544.32, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "al foxes l" }
+{ "l_orderkey": 5831, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34892.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously even requests" }
+{ "l_orderkey": 5958, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21689.92, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "regular requests. bold, bold deposits unwin" }
+{ "l_orderkey": 5959, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44322.88, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-07-24", "l_receiptdate": "1992-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deposits. slyly special cou" }
+{ "l_orderkey": 322, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45313.92, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites detect qu" }
+{ "l_orderkey": 739, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the theodolites sn" }
+{ "l_orderkey": 838, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16992.72, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "hely unusual foxes. furio" }
+{ "l_orderkey": 932, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38705.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes. ironic pl" }
+{ "l_orderkey": 1024, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26433.12, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-04", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-03-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e blithely regular pi" }
+{ "l_orderkey": 1153, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-07-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " theodolites" }
+{ "l_orderkey": 1350, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30209.28, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ic, final " }
+{ "l_orderkey": 1600, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45313.92, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously silent foxes could wake. car" }
+{ "l_orderkey": 2147, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32097.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "egular deposits hang car" }
+{ "l_orderkey": 3073, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10384.44, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions sleep according to the " }
+{ "l_orderkey": 3138, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25489.08, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "counts cajole fluffily carefully special i" }
+{ "l_orderkey": 3138, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-19", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites around the carefully busy the" }
+{ "l_orderkey": 3269, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 36817.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "he express packages?" }
+{ "l_orderkey": 3621, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18880.8, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gular accounts use carefully with" }
+{ "l_orderkey": 4292, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20768.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-02-16", "l_receiptdate": "1992-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "refully expres" }
+{ "l_orderkey": 4961, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35873.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e on the blithely bold accounts. unu" }
+{ "l_orderkey": 5250, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1888.08, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-10-10", "l_receiptdate": "1995-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its. final pinto" }
+{ "l_orderkey": 5284, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22656.96, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle according " }
+{ "l_orderkey": 5319, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36817.56, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-06-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts. furiously silent" }
+{ "l_orderkey": 5381, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 29265.24, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the carefully expre" }
+{ "l_orderkey": 5568, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16992.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-08-18", "l_receiptdate": "1995-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "structions haggle. carefully regular " }
+{ "l_orderkey": 5728, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-25", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nd the bravely final deposits. final ideas" }
+{ "l_orderkey": 5859, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8496.36, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-06-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ges boost quickly. blithely r" }
+{ "l_orderkey": 32, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1890.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " express accounts wake according to the" }
+{ "l_orderkey": 98, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13230.56, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1994-11-22", "l_receiptdate": "1995-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " cajole furiously. blithely ironic ideas " }
+{ "l_orderkey": 131, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ending requests. final, ironic pearls slee" }
+{ "l_orderkey": 417, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 38746.64, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "tes. regular requests across the " }
+{ "l_orderkey": 517, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26461.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-18", "l_receiptdate": "1997-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " requests. special, fi" }
+{ "l_orderkey": 643, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 36856.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " the pains. carefully s" }
+{ "l_orderkey": 930, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-21", "l_commitdate": "1995-02-20", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quickly regular pinto beans sle" }
+{ "l_orderkey": 1158, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4725.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "symptotes along the care" }
+{ "l_orderkey": 1893, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2835.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gular, even ideas. fluffily bol" }
+{ "l_orderkey": 2055, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14175.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-15", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-10-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "furiously bold " }
+{ "l_orderkey": 2278, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic pinto beans br" }
+{ "l_orderkey": 2278, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "into beans. blit" }
+{ "l_orderkey": 2337, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46306.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " along the packages. furiously p" }
+{ "l_orderkey": 2720, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4725.2, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-24", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-07-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously ironic foxes thrash" }
+{ "l_orderkey": 2818, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10395.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ggle across the carefully blithe" }
+{ "l_orderkey": 3009, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45361.92, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " dependencies sleep quickly a" }
+{ "l_orderkey": 3105, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8505.36, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "es wake among t" }
+{ "l_orderkey": 3239, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-09", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-02-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "d blithely stea" }
+{ "l_orderkey": 3239, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40636.72, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y. bold pinto beans use " }
+{ "l_orderkey": 3460, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 26461.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "inal, ironic instructions. carefully" }
+{ "l_orderkey": 3686, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29296.24, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-09", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle across the courts. furiously regu" }
+{ "l_orderkey": 3879, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33076.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-23", "l_receiptdate": "1995-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "o beans. accounts cajole furiously. re" }
+{ "l_orderkey": 4515, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20790.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le quickly above the even, bold ideas." }
+{ "l_orderkey": 4518, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17955.76, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ter the slyly bo" }
+{ "l_orderkey": 4547, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14175.6, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-12-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e carefully across the unus" }
+{ "l_orderkey": 4548, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tions integrat" }
+{ "l_orderkey": 4935, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 46306.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-08-21", "l_receiptdate": "1993-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ffily after the furiou" }
+{ "l_orderkey": 4960, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5670.24, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-21", "l_commitdate": "1995-05-13", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ual package" }
+{ "l_orderkey": 5092, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32131.36, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ckages nag " }
+{ "l_orderkey": 5122, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11340.48, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-02", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar instructions " }
+{ "l_orderkey": 5158, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 40636.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual platelets. slyly even foxes cajole " }
+{ "l_orderkey": 5223, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22680.96, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "refully bold courts besides the regular," }
+{ "l_orderkey": 5507, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3780.16, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "into beans are" }
+{ "l_orderkey": 5537, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9450.4, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " sleep carefully slyly bold depos" }
+{ "l_orderkey": 70, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-03-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "alongside of the deposits. fur" }
+{ "l_orderkey": 166, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1995-11-29", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e carefully bold " }
+{ "l_orderkey": 356, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3784.16, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the dependencies nod unusual, final ac" }
+{ "l_orderkey": 901, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1892.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-25", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d foxes use slyly" }
+{ "l_orderkey": 1031, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14190.6, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-07", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "about the carefully bold a" }
+{ "l_orderkey": 1120, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20812.88, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1998-01-25", "l_receiptdate": "1997-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ons. slyly silent requests sleep silent" }
+{ "l_orderkey": 1318, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24597.04, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-26", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly. regular, u" }
+{ "l_orderkey": 1859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-06-05", "l_receiptdate": "1997-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ffily ironic pac" }
+{ "l_orderkey": 2370, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2838.12, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly regular Tiresia" }
+{ "l_orderkey": 2435, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16082.68, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cajole aft" }
+{ "l_orderkey": 2499, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45409.92, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic ideas cajole quickly requests. caref" }
+{ "l_orderkey": 2503, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47302.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s around the slyly " }
+{ "l_orderkey": 2560, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29327.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-14", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "to beans. blithely regular Tiresias int" }
+{ "l_orderkey": 2919, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41625.76, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final ideas haggle carefully fluff" }
+{ "l_orderkey": 3043, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21758.92, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uickly above the pending," }
+{ "l_orderkey": 3141, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44463.88, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1996-01-13", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " are slyly pi" }
+{ "l_orderkey": 3173, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15136.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-12", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e special," }
+{ "l_orderkey": 3201, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-27", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-10-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing to the furiously expr" }
+{ "l_orderkey": 3525, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar excuses wake carefull" }
+{ "l_orderkey": 3648, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32165.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " deposits are furiously. careful, " }
+{ "l_orderkey": 3684, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5676.24, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he silent requests. packages sleep fu" }
+{ "l_orderkey": 3779, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26489.12, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s. close requests sleep" }
+{ "l_orderkey": 3840, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-10-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress pinto beans. accounts a" }
+{ "l_orderkey": 3845, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 946.04, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " blithely ironic t" }
+{ "l_orderkey": 4230, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35949.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly regular packages. regular ideas boost" }
+{ "l_orderkey": 4320, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26489.12, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "nts. even, ironic excuses hagg" }
+{ "l_orderkey": 4322, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16082.68, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-05-31", "l_receiptdate": "1998-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ructions boost " }
+{ "l_orderkey": 4514, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14190.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "! unusual, special deposits afte" }
+{ "l_orderkey": 4676, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-10-18", "l_receiptdate": "1996-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "cuses boost above" }
+{ "l_orderkey": 4711, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17028.72, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " blithely. bold asymptote" }
+{ "l_orderkey": 5633, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25543.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-28", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ructions. even ideas haggle carefully r" }
+{ "l_orderkey": 5665, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44463.88, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-09-19", "l_receiptdate": "1993-11-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s mold fluffily. final deposits along the" }
+{ "l_orderkey": 5767, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34057.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ake carefully. packages " }
+{ "l_orderkey": 5859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31219.32, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eposits unwind furiously final pinto bea" }
+{ "l_orderkey": 100, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43563.84, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-10", "l_receiptdate": "1998-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ular accounts. even" }
+{ "l_orderkey": 289, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts. quickly bold deposits alongside" }
+{ "l_orderkey": 486, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 43563.84, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-18", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "theodolites eat carefully furious" }
+{ "l_orderkey": 519, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25570.08, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-20", "l_commitdate": "1997-12-06", "l_receiptdate": "1997-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "le. even, final dependencies" }
+{ "l_orderkey": 768, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 44510.88, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "foxes. slyly ironic deposits a" }
+{ "l_orderkey": 805, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11364.48, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular foxes. furio" }
+{ "l_orderkey": 869, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ong the furiously bold instructi" }
+{ "l_orderkey": 899, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23676.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rly final sentiments. bold pinto beans " }
+{ "l_orderkey": 1156, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 18940.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "deposits sleep bravel" }
+{ "l_orderkey": 1184, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25570.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-10", "l_commitdate": "1997-12-02", "l_receiptdate": "1998-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s wake fluffily. fl" }
+{ "l_orderkey": 1665, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely final requests. requests" }
+{ "l_orderkey": 2311, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 947.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-06-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ptotes. furiously regular theodolite" }
+{ "l_orderkey": 2341, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11364.48, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-06", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". quickly final deposits sl" }
+{ "l_orderkey": 2375, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24623.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "rate across the" }
+{ "l_orderkey": 2465, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 47352.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the pending th" }
+{ "l_orderkey": 3105, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28411.2, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-03-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts boost among t" }
+{ "l_orderkey": 3171, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 32199.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r the final, even packages. quickly" }
+{ "l_orderkey": 3682, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16099.68, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", ironic packages wake a" }
+{ "l_orderkey": 3685, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35040.48, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ress attai" }
+{ "l_orderkey": 4037, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-12", "l_receiptdate": "1993-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s around the blithely ironic ac" }
+{ "l_orderkey": 4194, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17046.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1994-12-04", "l_receiptdate": "1995-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ld packages. quickly eve" }
+{ "l_orderkey": 4387, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8523.36, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-04", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c ideas. slyly regular packages sol" }
+{ "l_orderkey": 4421, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 41669.76, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-06-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "le carefully. bl" }
+{ "l_orderkey": 4548, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16099.68, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-23", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y ironic requests above the fluffily d" }
+{ "l_orderkey": 4608, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47352.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-09-01", "l_receiptdate": "1994-08-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " theodolites" }
+{ "l_orderkey": 4609, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26517.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously. quickly final requests cajole fl" }
+{ "l_orderkey": 4769, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-08-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". slyly even deposit" }
+{ "l_orderkey": 5636, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-11", "l_commitdate": "1995-04-27", "l_receiptdate": "1995-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en, fluffy accounts amon" }
+{ "l_orderkey": 5637, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13258.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits wak" }
+{ "l_orderkey": 5639, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10417.44, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the unusual pinto beans caj" }
+{ "l_orderkey": 5761, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38828.64, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "pecial deposits. qu" }
+{ "l_orderkey": 420, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42661.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " final accounts. furiously express forges" }
+{ "l_orderkey": 832, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22752.96, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-06-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully. carefully speci" }
+{ "l_orderkey": 928, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22752.96, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s the furiously regular warthogs im" }
+{ "l_orderkey": 992, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "fily. quickly special deposit" }
+{ "l_orderkey": 997, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16116.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle quickly furiously" }
+{ "l_orderkey": 1157, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7584.32, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-03-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely even pa" }
+{ "l_orderkey": 1218, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41713.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-05", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "thely ironic accounts wake slyly" }
+{ "l_orderkey": 1574, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s. slyly regular depen" }
+{ "l_orderkey": 1634, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "counts alo" }
+{ "l_orderkey": 1667, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5688.24, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-21", "l_commitdate": "1997-12-19", "l_receiptdate": "1998-01-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " nag quickly above th" }
+{ "l_orderkey": 1793, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27493.16, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ar excuses. " }
+{ "l_orderkey": 1831, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ent deposits. regular saute" }
+{ "l_orderkey": 2050, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 25.0, "l_extendedprice": 23701.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y according to " }
+{ "l_orderkey": 2211, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23701.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas. carefully special theodolites along" }
+{ "l_orderkey": 2213, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully pend" }
+{ "l_orderkey": 2304, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l excuses after the ev" }
+{ "l_orderkey": 2503, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s cajole. slyly close courts nod f" }
+{ "l_orderkey": 2660, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16116.68, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans wake after the furious" }
+{ "l_orderkey": 2691, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1896.08, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-10", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s cajole at the blithely ironic warthog" }
+{ "l_orderkey": 2753, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37921.6, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "latelets kindle slyly final depos" }
+{ "l_orderkey": 2785, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32233.36, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kages wake carefully silent " }
+{ "l_orderkey": 3042, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18012.76, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-05", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e carefully. regul" }
+{ "l_orderkey": 3104, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44557.88, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-25", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ily daring acc" }
+{ "l_orderkey": 3585, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31285.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-14", "l_commitdate": "1995-01-19", "l_receiptdate": "1994-12-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ironic dependencies serve furi" }
+{ "l_orderkey": 3619, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43609.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "press, expres" }
+{ "l_orderkey": 3937, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28441.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-17", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al packages slee" }
+{ "l_orderkey": 4192, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45505.92, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-07-11", "l_receiptdate": "1998-09-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ests. quickly bol" }
+{ "l_orderkey": 4324, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11376.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-10-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c packages. furiously express sauternes" }
+{ "l_orderkey": 4422, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " theodolites shal" }
+{ "l_orderkey": 4836, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15168.64, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-03-05", "l_receiptdate": "1997-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gular packages against the express reque" }
+{ "l_orderkey": 4870, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46453.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-14", "l_commitdate": "1994-10-24", "l_receiptdate": "1994-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular packages " }
+{ "l_orderkey": 5056, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6636.28, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-28", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rouches after the pending instruc" }
+{ "l_orderkey": 5090, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-29", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly express accounts. slyly even r" }
+{ "l_orderkey": 5155, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 948.04, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "oze slyly after the silent, regular idea" }
+{ "l_orderkey": 5473, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " excuses sleep blithely! regular dep" }
+{ "l_orderkey": 5474, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29389.24, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously express ideas. speci" }
+{ "l_orderkey": 5476, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12324.52, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special ac" }
+{ "l_orderkey": 455, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42706.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-20", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "thrash ironically regular packages. qui" }
+{ "l_orderkey": 739, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11388.48, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-20", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-08-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "le slyly along the close i" }
+{ "l_orderkey": 768, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 31318.32, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sly ironic instructions. excuses can hagg" }
+{ "l_orderkey": 898, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10439.44, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "etly bold accounts " }
+{ "l_orderkey": 933, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21827.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the furiously bold dinos. sly" }
+{ "l_orderkey": 1157, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15184.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-09", "l_receiptdate": "1998-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tions hang" }
+{ "l_orderkey": 1191, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27522.16, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular pin" }
+{ "l_orderkey": 1220, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23726.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages affi" }
+{ "l_orderkey": 1286, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45553.92, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-07-11", "l_receiptdate": "1993-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unts alongs" }
+{ "l_orderkey": 1543, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2847.12, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-29", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "sleep along the furiou" }
+{ "l_orderkey": 1569, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-31", "l_receiptdate": "1998-06-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " instructions." }
+{ "l_orderkey": 1761, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 35114.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular packages wake after" }
+{ "l_orderkey": 2022, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45553.92, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "counts. slyly enticing accounts are during " }
+{ "l_orderkey": 2050, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27522.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "oxes alongsid" }
+{ "l_orderkey": 2115, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44604.88, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-29", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular accounts integrate brav" }
+{ "l_orderkey": 2149, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 44604.88, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely final depo" }
+{ "l_orderkey": 2435, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "alongside of the s" }
+{ "l_orderkey": 2464, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9490.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-04", "l_commitdate": "1997-12-29", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "slyly final pinto bean" }
+{ "l_orderkey": 2983, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10439.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-02-27", "l_receiptdate": "1992-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "aids integrate s" }
+{ "l_orderkey": 3396, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "cial packages cajole blithely around the " }
+{ "l_orderkey": 3426, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 29420.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-11", "l_commitdate": "1996-12-10", "l_receiptdate": "1996-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " even sentiment" }
+{ "l_orderkey": 3653, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1898.08, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "n accounts. fina" }
+{ "l_orderkey": 3683, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38910.64, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ress instructions. slyly express a" }
+{ "l_orderkey": 4225, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23726.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-10", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "se fluffily. busily ironic requests are;" }
+{ "l_orderkey": 4486, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18031.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-28", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pending foxes after" }
+{ "l_orderkey": 4711, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14235.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-03", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ld requests: furiously final inst" }
+{ "l_orderkey": 4771, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8541.36, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-19", "l_receiptdate": "1993-03-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "riously after the packages. fina" }
+{ "l_orderkey": 5157, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11388.48, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-07", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es. busily " }
+{ "l_orderkey": 97, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35151.85, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic requests boost carefully quic" }
+{ "l_orderkey": 354, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quickly regular grouches will eat. careful" }
+{ "l_orderkey": 644, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21851.15, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uctions nag quickly alongside of t" }
+{ "l_orderkey": 870, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34201.8, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-18", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-11-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "fily. furiously final accounts are " }
+{ "l_orderkey": 1089, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33251.75, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-08-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly express deposits haggle" }
+{ "l_orderkey": 1124, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23751.25, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-10-14", "l_receiptdate": "1998-08-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ggle slyly according" }
+{ "l_orderkey": 1281, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3800.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ggle against the even requests. requests " }
+{ "l_orderkey": 1475, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11400.6, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1997-12-30", "l_receiptdate": "1998-01-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully-- excuses sublate" }
+{ "l_orderkey": 1506, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 36101.9, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-19", "l_receiptdate": "1992-12-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "xpress, regular excuse" }
+{ "l_orderkey": 1762, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37051.95, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-11-09", "l_receiptdate": "1994-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic platelets sleep along t" }
+{ "l_orderkey": 2406, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15200.8, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-11-28", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " special accou" }
+{ "l_orderkey": 2432, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28501.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " requests wake alongside of" }
+{ "l_orderkey": 2562, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-15", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar pinto beans. blithely ev" }
+{ "l_orderkey": 2724, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20901.1, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-10-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "express fo" }
+{ "l_orderkey": 2753, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-08", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " carefully bold deposits sublate s" }
+{ "l_orderkey": 2786, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40852.15, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ons. theodolites after" }
+{ "l_orderkey": 2885, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 38002.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " express depos" }
+{ "l_orderkey": 3458, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43702.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-08", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nod across the boldly even instruct" }
+{ "l_orderkey": 3522, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-29", "l_commitdate": "1994-12-15", "l_receiptdate": "1994-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ic tithes. car" }
+{ "l_orderkey": 3523, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22801.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke according to the doggedly re" }
+{ "l_orderkey": 3841, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8550.45, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-12-26", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s according to the courts shall nag s" }
+{ "l_orderkey": 3877, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11400.6, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-08-09", "l_receiptdate": "1993-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal requests. even requests are. pac" }
+{ "l_orderkey": 3943, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully regular deposits accord" }
+{ "l_orderkey": 4034, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fully around the furiously ironic re" }
+{ "l_orderkey": 4131, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5700.3, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ns cajole slyly. even, iro" }
+{ "l_orderkey": 4324, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-10-08", "l_receiptdate": "1995-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " express ideas. blithely blit" }
+{ "l_orderkey": 4581, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6650.35, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express accounts d" }
+{ "l_orderkey": 4612, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-09", "l_commitdate": "1993-11-08", "l_receiptdate": "1994-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "equests haggle carefully silent excus" }
+{ "l_orderkey": 4645, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42752.25, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ular ideas. slyly" }
+{ "l_orderkey": 4706, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans. finally special instruct" }
+{ "l_orderkey": 4865, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19951.05, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-10", "l_receiptdate": "1997-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "eposits detect sly" }
+{ "l_orderkey": 4967, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14250.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. blithel" }
+{ "l_orderkey": 5031, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14250.75, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly pending theodolites." }
+{ "l_orderkey": 5249, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29451.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "f the excuses. furiously fin" }
+{ "l_orderkey": 5347, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17100.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he ideas among the requests " }
+{ "l_orderkey": 5410, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7600.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-12", "l_commitdate": "1998-10-22", "l_receiptdate": "1998-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly. fluffily ironic platelets alon" }
+{ "l_orderkey": 5573, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1900.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " even foxes. specia" }
+{ "l_orderkey": 5925, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45602.4, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-19", "l_receiptdate": "1996-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle after the fo" }
+{ "l_orderkey": 5926, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic requests" }
+{ "l_orderkey": 224, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3804.2, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-08", "l_commitdate": "1994-08-24", "l_receiptdate": "1994-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions " }
+{ "l_orderkey": 288, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "instructions wa" }
+{ "l_orderkey": 512, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e slyly silent accounts serve with" }
+{ "l_orderkey": 582, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46601.45, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nts according to the furiously regular pin" }
+{ "l_orderkey": 643, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45650.4, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-10", "l_commitdate": "1995-06-07", "l_receiptdate": "1995-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly ironic accounts" }
+{ "l_orderkey": 644, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36139.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages. blithely slow accounts nag quic" }
+{ "l_orderkey": 676, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8559.45, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-03", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-04-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "aintain sl" }
+{ "l_orderkey": 1189, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21874.15, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-08-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. fluffy Tiresias run quickly. bra" }
+{ "l_orderkey": 1347, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pin" }
+{ "l_orderkey": 1508, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15216.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-05-30", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously across the ironic, unusua" }
+{ "l_orderkey": 1731, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly slyly speci" }
+{ "l_orderkey": 1920, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lly. ideas wa" }
+{ "l_orderkey": 1926, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22825.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-04", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e theodolites." }
+{ "l_orderkey": 2115, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2853.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quickly ironic dolphin" }
+{ "l_orderkey": 2144, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43748.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes haggle blithel" }
+{ "l_orderkey": 2244, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2853.15, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-03-15", "l_receiptdate": "1993-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " beans for the regular platel" }
+{ "l_orderkey": 2305, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6657.35, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-15", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gular deposits boost about the foxe" }
+{ "l_orderkey": 2465, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32335.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. regular package" }
+{ "l_orderkey": 2561, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13314.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-07", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep unusual, ironic accounts" }
+{ "l_orderkey": 2690, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-13", "l_commitdate": "1996-05-22", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " doubt careful" }
+{ "l_orderkey": 2786, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39944.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-15", "l_commitdate": "1992-04-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "unts are against the furious" }
+{ "l_orderkey": 2951, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14265.75, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-25", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "inal account" }
+{ "l_orderkey": 3104, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19021.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-24", "l_receiptdate": "1994-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s are. furiously s" }
+{ "l_orderkey": 3233, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21874.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1995-01-11", "l_receiptdate": "1994-12-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pending instructions use after the carefu" }
+{ "l_orderkey": 3972, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-24", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y final theodolite" }
+{ "l_orderkey": 4099, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34237.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-06", "l_commitdate": "1992-09-28", "l_receiptdate": "1992-12-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans cajole slyly quickly ironic " }
+{ "l_orderkey": 4193, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27580.45, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. final packages use blit" }
+{ "l_orderkey": 4324, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41846.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ainst the u" }
+{ "l_orderkey": 4354, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1994-12-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s nag quickly " }
+{ "l_orderkey": 4544, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-09-07", "l_receiptdate": "1997-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular packages. s" }
+{ "l_orderkey": 4741, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "t, regular requests" }
+{ "l_orderkey": 4836, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11412.6, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sly ironic accoun" }
+{ "l_orderkey": 4932, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12363.65, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-10-16", "l_receiptdate": "1993-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "slyly according to the furiously fin" }
+{ "l_orderkey": 5088, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38993.05, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-07", "l_receiptdate": "1993-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ing requests. " }
+{ "l_orderkey": 5860, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ual patterns try to eat carefully above" }
+{ "l_orderkey": 928, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 40938.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-14", "l_commitdate": "1995-04-21", "l_receiptdate": "1995-05-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely. express, silent requests doze at" }
+{ "l_orderkey": 1057, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18088.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r-- packages haggle alon" }
+{ "l_orderkey": 1280, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22849.2, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y pending orbits boost after the slyly" }
+{ "l_orderkey": 1313, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45698.4, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-20", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s are quick" }
+{ "l_orderkey": 1571, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1993-02-24", "l_receiptdate": "1993-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ng to the fluffily unusual " }
+{ "l_orderkey": 1637, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19993.05, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-30", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly ironic theodolites use b" }
+{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31417.65, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-01-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. excuses a" }
+{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35225.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-17", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " integrate. quickly unusual" }
+{ "l_orderkey": 1767, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 38082.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ep. accounts nag blithely fu" }
+{ "l_orderkey": 1799, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7616.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ealms upon the special, ironic waters" }
+{ "l_orderkey": 2019, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "are carefully furiously regular requ" }
+{ "l_orderkey": 2183, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23801.25, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-08-21", "l_receiptdate": "1996-08-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he quickly f" }
+{ "l_orderkey": 2279, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " above the furiously ironic deposits. " }
+{ "l_orderkey": 2338, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28561.5, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ould have to nag quickly" }
+{ "l_orderkey": 2567, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5712.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-05-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole regular, final acco" }
+{ "l_orderkey": 2945, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests use" }
+{ "l_orderkey": 3040, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40938.15, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts nag slyly alongside of the depos" }
+{ "l_orderkey": 3075, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1904.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". unusual, unusual accounts haggle furious" }
+{ "l_orderkey": 3106, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nstructions wake. furiously " }
+{ "l_orderkey": 3111, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9520.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng the slyly ironic inst" }
+{ "l_orderkey": 3430, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 21897.15, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-03-01", "l_receiptdate": "1995-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "eas according to the" }
+{ "l_orderkey": 3907, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42842.25, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " about the regular pac" }
+{ "l_orderkey": 3969, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37129.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-06-13", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly bold ideas s" }
+{ "l_orderkey": 4003, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-02", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ar grouches s" }
+{ "l_orderkey": 4388, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12376.65, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly even, expre" }
+{ "l_orderkey": 4448, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22849.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-09", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal packages along the ironic instructi" }
+{ "l_orderkey": 4454, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45698.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-23", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans wake across th" }
+{ "l_orderkey": 4611, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously. furiously regular" }
+{ "l_orderkey": 4838, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24753.3, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular requests boost about the packages. r" }
+{ "l_orderkey": 4934, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven, ironic ideas" }
+{ "l_orderkey": 5159, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4760.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal deposits. pending, ironic ideas grow" }
+{ "l_orderkey": 5282, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30465.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-01", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-03-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "onic deposits; furiou" }
+{ "l_orderkey": 480, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20967.1, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-07-28", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "into beans cajole furiously. accounts s" }
+{ "l_orderkey": 772, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33356.75, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "kly thin packages wake slowly" }
+{ "l_orderkey": 1060, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 953.05, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits detect carefully abo" }
+{ "l_orderkey": 1698, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35262.85, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular ideas. deposit" }
+{ "l_orderkey": 1731, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35262.85, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-30", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " beans use furiously slyly b" }
+{ "l_orderkey": 1888, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45746.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ar ideas cajole. regular p" }
+{ "l_orderkey": 1893, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5718.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ar accounts use. daringly ironic packag" }
+{ "l_orderkey": 1952, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6671.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-05-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "about the express, even requ" }
+{ "l_orderkey": 1984, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42887.25, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "p. quickly final ideas sle" }
+{ "l_orderkey": 1991, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46699.45, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-11-29", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nd the ideas affi" }
+{ "l_orderkey": 2240, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37168.95, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-06-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y orbits. final depos" }
+{ "l_orderkey": 2246, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20967.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-08-03", "l_receiptdate": "1996-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ructions wake carefully fina" }
+{ "l_orderkey": 2400, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21920.15, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-08-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tions. fluffily ironic platelets cajole c" }
+{ "l_orderkey": 2532, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "unusual sentiments. even pinto" }
+{ "l_orderkey": 2562, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26685.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-04", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ans haggle special, special packages. " }
+{ "l_orderkey": 2950, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13342.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ccounts haggle carefully according " }
+{ "l_orderkey": 3717, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nside the regular packages sleep" }
+{ "l_orderkey": 4036, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20014.05, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-28", "l_receiptdate": "1997-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e carefully. qui" }
+{ "l_orderkey": 4800, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22873.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-14", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully carefully r" }
+{ "l_orderkey": 4967, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40981.15, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ons. slyly ironic requests" }
+{ "l_orderkey": 5153, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13342.7, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " slyly daring pinto beans lose blithely fi" }
+{ "l_orderkey": 5664, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29544.55, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-10", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ainst the never silent request" }
+{ "l_orderkey": 5793, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19061.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-09-04", "l_receiptdate": "1997-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e carefully ex" }
+{ "l_orderkey": 5924, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46699.45, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-25", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "inly final excuses. blithely regular requ" }
+{ "l_orderkey": 100, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35299.85, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nd the quickly s" }
+{ "l_orderkey": 385, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43886.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-29", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lthily ironic f" }
+{ "l_orderkey": 642, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24805.3, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests according to the unu" }
+{ "l_orderkey": 770, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-23", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits dazzle fluffily alongside of " }
+{ "l_orderkey": 772, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 40070.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-16", "l_receiptdate": "1993-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express foxes abo" }
+{ "l_orderkey": 803, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7632.4, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-08-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ronic theodo" }
+{ "l_orderkey": 833, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 954.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-04-05", "l_receiptdate": "1994-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ffily ironic theodolites" }
+{ "l_orderkey": 1253, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12402.65, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-03-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "al packages" }
+{ "l_orderkey": 1346, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12402.65, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-22", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "arefully brave deposits into the slyly iro" }
+{ "l_orderkey": 1350, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20035.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-12-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lyly above the evenly " }
+{ "l_orderkey": 1473, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47702.5, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-05-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "requests wake express deposits. special, ir" }
+{ "l_orderkey": 1701, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1908.1, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ween the pending, final accounts. " }
+{ "l_orderkey": 1988, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7632.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-20", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "le quickly ac" }
+{ "l_orderkey": 2150, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37207.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ess accounts nag. unusual asymptotes haggl" }
+{ "l_orderkey": 2311, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14310.75, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-06", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ve the blithely pending accounts. furio" }
+{ "l_orderkey": 2533, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34345.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-07-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ss requests sleep neve" }
+{ "l_orderkey": 2535, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4770.25, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " across the express requests. silent, eve" }
+{ "l_orderkey": 2784, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21943.15, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests lose after " }
+{ "l_orderkey": 3046, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43886.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits sleep furious" }
+{ "l_orderkey": 3111, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13356.7, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "re. pinto " }
+{ "l_orderkey": 3271, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17172.9, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-28", "l_receiptdate": "1992-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages eat around the furiously regul" }
+{ "l_orderkey": 3427, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39116.05, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s the carefully" }
+{ "l_orderkey": 3749, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9540.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "essly. regular pi" }
+{ "l_orderkey": 3968, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25759.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t silently." }
+{ "l_orderkey": 4034, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41024.15, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1994-01-08", "l_receiptdate": "1993-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits wake carefully af" }
+{ "l_orderkey": 4321, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 42932.25, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " haggle ironically bold theodolites. quick" }
+{ "l_orderkey": 4515, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 30529.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-07", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully express depo" }
+{ "l_orderkey": 4645, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-12-11", "l_receiptdate": "1994-11-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "braids. ironic dependencies main" }
+{ "l_orderkey": 4865, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31483.65, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y pending notornis ab" }
+{ "l_orderkey": 5350, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11448.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1993-11-21", "l_receiptdate": "1994-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " cajole. even instructions haggle. blithe" }
+{ "l_orderkey": 5408, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45794.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular " }
+{ "l_orderkey": 5412, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1908.1, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " sleep above the furiou" }
+{ "l_orderkey": 5701, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16218.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-04-08", "l_receiptdate": "1997-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tes. quickly final a" }
+{ "l_orderkey": 5925, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28621.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " the packa" }
+{ "l_orderkey": 39, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 41067.15, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-17", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "yly regular i" }
+{ "l_orderkey": 871, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44887.35, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-01", "l_receiptdate": "1996-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss, final dep" }
+{ "l_orderkey": 928, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 47752.5, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " slyly slyly special request" }
+{ "l_orderkey": 964, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 42022.2, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-10-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ronic deposit" }
+{ "l_orderkey": 1408, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-03-14", "l_receiptdate": "1998-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ic foxes ca" }
+{ "l_orderkey": 1574, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23876.25, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-16", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly silent accounts." }
+{ "l_orderkey": 1856, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9550.5, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-11", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he furiously even theodolites. account" }
+{ "l_orderkey": 2022, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions dazzle carefull" }
+{ "l_orderkey": 2177, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32471.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-03", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tes are doggedly quickly" }
+{ "l_orderkey": 2180, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nic instructions haggle careful" }
+{ "l_orderkey": 2181, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26741.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-23", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s excuses sleep car" }
+{ "l_orderkey": 2657, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10505.55, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly enticing requests. fur" }
+{ "l_orderkey": 2849, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-06-05", "l_receiptdate": "1996-05-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "mong the carefully regular theodol" }
+{ "l_orderkey": 3399, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7640.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s use carefully carefully ir" }
+{ "l_orderkey": 3588, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26741.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " express sheaves. unusual theodo" }
+{ "l_orderkey": 4006, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10505.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ress foxes cajole quick" }
+{ "l_orderkey": 4519, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28651.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-11", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "totes. slyly bold somas after the " }
+{ "l_orderkey": 4672, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42977.25, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " platelets use amon" }
+{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully ironic dep" }
+{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40112.1, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-28", "l_receiptdate": "1992-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sts. blithely regular deposits will have" }
+{ "l_orderkey": 5124, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41067.15, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-10", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "onic package" }
+{ "l_orderkey": 5157, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33426.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously sil" }
+{ "l_orderkey": 5186, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y ruthless foxes. fluffily " }
+{ "l_orderkey": 5382, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12415.65, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-12", "l_receiptdate": "1992-02-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eodolites. final foxes " }
+{ "l_orderkey": 5569, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the fluffily" }
+{ "l_orderkey": 5605, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 42977.25, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly unusual instructions. carefully ironic p" }
+{ "l_orderkey": 5697, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22921.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-27", "l_commitdate": "1992-11-28", "l_receiptdate": "1992-11-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uffily iro" }
+{ "l_orderkey": 5699, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake fluffily u" }
+{ "l_orderkey": 5699, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 43932.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-28", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-12-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "o the slyly" }
+{ "l_orderkey": 5956, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21966.15, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly slyly special " }
+{ "l_orderkey": 70, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-02-17", "l_receiptdate": "1994-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages wake pending accounts." }
+{ "l_orderkey": 356, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 39198.05, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-28", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " according to the express foxes will" }
+{ "l_orderkey": 387, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-21", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular dependencies" }
+{ "l_orderkey": 708, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c pinto beans nag after the account" }
+{ "l_orderkey": 903, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8604.45, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he slyly ev" }
+{ "l_orderkey": 1248, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24857.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " ironic dependen" }
+{ "l_orderkey": 1638, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-27", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " quickly expres" }
+{ "l_orderkey": 2272, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34417.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ironic packages; quickly iron" }
+{ "l_orderkey": 2308, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34417.8, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-11-27", "l_receiptdate": "1992-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ong the pending hockey players. blithe" }
+{ "l_orderkey": 2531, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26769.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-07-31", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its. busily" }
+{ "l_orderkey": 2752, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3824.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-01-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "telets haggle. regular, final " }
+{ "l_orderkey": 3205, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9560.5, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-07-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " deposits cajole careful" }
+{ "l_orderkey": 3590, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24857.3, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully along th" }
+{ "l_orderkey": 3680, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31549.65, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-16", "l_commitdate": "1993-02-19", "l_receiptdate": "1993-04-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts. ironic, fina" }
+{ "l_orderkey": 3685, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35373.85, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-03-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully sly requests are regular, regu" }
+{ "l_orderkey": 4129, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30593.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ckages haggl" }
+{ "l_orderkey": 4421, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43978.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "reful packages. bold, " }
+{ "l_orderkey": 4450, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5736.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-09-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "eposits. foxes cajole unusual fox" }
+{ "l_orderkey": 4705, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15296.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special ideas nag sl" }
+{ "l_orderkey": 4996, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33461.75, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. unusual, regular dolphins integrate care" }
+{ "l_orderkey": 5190, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41110.15, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies use fluffily unusual requests? hoc" }
+{ "l_orderkey": 5347, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5736.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-05-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lly unusual ideas. sl" }
+{ "l_orderkey": 5411, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-12", "l_commitdate": "1997-08-03", "l_receiptdate": "1997-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " bold, ironic theodo" }
+{ "l_orderkey": 5697, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40154.1, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-12-08", "l_receiptdate": "1993-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "inal theodolites cajole after the bli" }
+{ "l_orderkey": 194, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7656.4, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uriously unusual excuses" }
+{ "l_orderkey": 198, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully caref" }
+{ "l_orderkey": 231, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-05", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously special decoys wake q" }
+{ "l_orderkey": 450, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 38282.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-04-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ve. asymptote" }
+{ "l_orderkey": 548, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " engage quickly. regular theo" }
+{ "l_orderkey": 582, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6699.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-29", "l_receiptdate": "1997-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ithely unusual t" }
+{ "l_orderkey": 736, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12441.65, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-16", "l_commitdate": "1998-07-26", "l_receiptdate": "1998-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "st furiously among the " }
+{ "l_orderkey": 805, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 27754.45, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-08-15", "l_receiptdate": "1995-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites according to the slyly f" }
+{ "l_orderkey": 837, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37324.95, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-08-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ecial pinto bea" }
+{ "l_orderkey": 962, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34453.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al foxes. iron" }
+{ "l_orderkey": 962, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19141.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-26", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits use fluffily according to " }
+{ "l_orderkey": 964, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 46895.45, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-08-10", "l_receiptdate": "1995-10-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ounts. blithely regular packag" }
+{ "l_orderkey": 1189, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21055.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "quickly unusual platelets lose forges. ca" }
+{ "l_orderkey": 1345, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-29", "l_receiptdate": "1992-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". slyly silent accounts sublat" }
+{ "l_orderkey": 1444, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32539.7, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. doggedly pend" }
+{ "l_orderkey": 1632, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-02-24", "l_receiptdate": "1997-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ructions! slyly" }
+{ "l_orderkey": 1664, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8613.45, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ges. fluffil" }
+{ "l_orderkey": 1924, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 38282.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ains sleep carefully" }
+{ "l_orderkey": 2146, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40196.1, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-11-02", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns according to the doggedly " }
+{ "l_orderkey": 2177, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 44024.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ending asymptotes." }
+{ "l_orderkey": 2215, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "against the carefu" }
+{ "l_orderkey": 2372, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 18183.95, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " beans haggle sometimes" }
+{ "l_orderkey": 2404, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18183.95, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cuses. quickly even in" }
+{ "l_orderkey": 2630, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7656.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "indle fluffily silent, ironic pi" }
+{ "l_orderkey": 3072, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5742.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular requests abov" }
+{ "l_orderkey": 3110, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-03-07", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "en deposits. ironic" }
+{ "l_orderkey": 3270, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27754.45, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-22", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ptotes nag above the quickly bold deposits" }
+{ "l_orderkey": 3271, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r the unusual Tiresia" }
+{ "l_orderkey": 3682, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he requests cajole quickly pending package" }
+{ "l_orderkey": 3778, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. blithely special theodoli" }
+{ "l_orderkey": 3975, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36367.9, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es are furiously: furi" }
+{ "l_orderkey": 4007, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30625.6, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-30", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nal accounts across t" }
+{ "l_orderkey": 4034, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44981.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "eodolites was slyly ironic ideas. de" }
+{ "l_orderkey": 4068, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ds wake carefully amon" }
+{ "l_orderkey": 4096, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16269.85, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-08-11", "l_receiptdate": "1992-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets alongside of the " }
+{ "l_orderkey": 4672, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12441.65, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1995-12-13", "l_receiptdate": "1996-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests? pending accounts against" }
+{ "l_orderkey": 5922, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37324.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e of the instructions. quick" }
+{ "l_orderkey": 935, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12454.65, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-13", "l_commitdate": "1997-11-30", "l_receiptdate": "1998-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ld platelet" }
+{ "l_orderkey": 1126, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-03-22", "l_receiptdate": "1998-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ons. final, unusual" }
+{ "l_orderkey": 1412, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35447.85, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "hely express excuses are " }
+{ "l_orderkey": 1542, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35447.85, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-15", "l_commitdate": "1993-10-17", "l_receiptdate": "1994-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e blithely unusual accounts. quic" }
+{ "l_orderkey": 2117, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41196.15, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " foxes sleep furiously " }
+{ "l_orderkey": 2208, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45986.4, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-13", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "sits. idly permanent request" }
+{ "l_orderkey": 2310, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34489.8, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-09", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "iously against the slyly special accounts" }
+{ "l_orderkey": 2501, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts. express, iron" }
+{ "l_orderkey": 2695, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15328.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "its. theodolites sleep slyly" }
+{ "l_orderkey": 3010, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar, even reques" }
+{ "l_orderkey": 3111, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28741.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eas are furiously slyly special deposits." }
+{ "l_orderkey": 3360, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3832.2, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly busy inst" }
+{ "l_orderkey": 3586, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1916.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts. slyly final ideas agai" }
+{ "l_orderkey": 3685, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-16", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. special asymptotes about the r" }
+{ "l_orderkey": 3751, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11496.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "accounts wake furious" }
+{ "l_orderkey": 4576, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 41196.15, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly final deposits. never" }
+{ "l_orderkey": 4678, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33531.75, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-27", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he accounts. fluffily bold sheaves b" }
+{ "l_orderkey": 4869, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-07", "l_receiptdate": "1994-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites cajole after the ideas. special t" }
+{ "l_orderkey": 4896, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5748.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-12", "l_receiptdate": "1992-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular deposits" }
+{ "l_orderkey": 4997, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-04-23", "l_receiptdate": "1998-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xpress, bo" }
+{ "l_orderkey": 5024, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 39280.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits hinder carefully " }
+{ "l_orderkey": 5282, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26825.4, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "fily final instruc" }
+{ "l_orderkey": 5569, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-07-22", "l_receiptdate": "1993-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "pitaphs. ironic req" }
+{ "l_orderkey": 5575, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. slyly pending theodolites prin" }
+{ "l_orderkey": 5698, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14370.75, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly ironic frets haggle carefully " }
+{ "l_orderkey": 5796, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25867.35, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s wake quickly aro" }
+{ "l_orderkey": 5857, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23951.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-12-17", "l_receiptdate": "1997-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding platelets. pending excu" }
+{ "l_orderkey": 5920, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-21", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully regular dolphins. furiousl" }
+{ "l_orderkey": 165, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " bold packages mainta" }
+{ "l_orderkey": 262, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 33566.75, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites cajole along the pending packag" }
+{ "l_orderkey": 354, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47952.5, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-05-20", "l_receiptdate": "1996-04-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "to beans s" }
+{ "l_orderkey": 674, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3836.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-05", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly express pinto beans sleep car" }
+{ "l_orderkey": 677, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30689.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-02-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly final" }
+{ "l_orderkey": 935, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " instructions. ironic acc" }
+{ "l_orderkey": 967, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39321.05, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ld foxes wake closely special" }
+{ "l_orderkey": 1249, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46993.45, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-28", "l_receiptdate": "1994-03-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ffily express theodo" }
+{ "l_orderkey": 1282, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18221.95, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-17", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nto beans. carefully close theodo" }
+{ "l_orderkey": 1284, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al packages use carefully express de" }
+{ "l_orderkey": 1510, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-12-05", "l_receiptdate": "1996-11-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he blithely regular req" }
+{ "l_orderkey": 1571, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17262.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1993-01-12", "l_receiptdate": "1993-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " pending grouches " }
+{ "l_orderkey": 1605, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37402.95, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-08-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nal dependencies-- quickly final frets acc" }
+{ "l_orderkey": 1667, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-12-01", "l_receiptdate": "1997-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "hrash final requests. care" }
+{ "l_orderkey": 1890, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "is wake carefully above the even id" }
+{ "l_orderkey": 2087, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5754.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "dazzle after the slyly si" }
+{ "l_orderkey": 2406, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1997-01-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "hely even foxes unwind furiously aga" }
+{ "l_orderkey": 2688, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-09", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "press, ironic excuses wake carefully id" }
+{ "l_orderkey": 2759, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9590.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. busily ironic theodo" }
+{ "l_orderkey": 2791, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46993.45, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-11-10", "l_receiptdate": "1995-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts sleep at the bold, regular pinto " }
+{ "l_orderkey": 2816, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-19", "l_commitdate": "1994-11-10", "l_receiptdate": "1994-11-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s; slyly even theodo" }
+{ "l_orderkey": 2945, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35484.85, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l instructions. regular, regular " }
+{ "l_orderkey": 3429, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans are fu" }
+{ "l_orderkey": 3521, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46034.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ses use. furiously express ideas wake f" }
+{ "l_orderkey": 3620, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39321.05, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "t attainments cajole qui" }
+{ "l_orderkey": 3845, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16303.85, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-12", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "counts haggle. reg" }
+{ "l_orderkey": 3906, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34525.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-07", "l_commitdate": "1992-08-08", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y. ironic deposits haggle sl" }
+{ "l_orderkey": 4099, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 37402.95, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-13", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fluffy accounts impress pending, iro" }
+{ "l_orderkey": 4672, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-03", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l instructions. blithely ironic packages " }
+{ "l_orderkey": 4998, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the blithely ironic " }
+{ "l_orderkey": 5157, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23976.25, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages detect. even requests against th" }
+{ "l_orderkey": 5444, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ut the courts cajole blithely excuses" }
+{ "l_orderkey": 5472, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-07", "l_receiptdate": "1993-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily pending attainments. unus" }
+{ "l_orderkey": 5569, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely bold requests boost fur" }
+{ "l_orderkey": 5696, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ter the instruct" }
+{ "l_orderkey": 5923, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 33566.75, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-21", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sts affix unusual, final requests. request" }
+{ "l_orderkey": 5957, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-23", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "platelets. furiously unusual requests " }
+{ "l_orderkey": 65, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24961.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-05-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pending deposits nag even packages. ca" }
+{ "l_orderkey": 294, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29761.86, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-06", "l_commitdate": "1993-08-19", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "le fluffily along the quick" }
+{ "l_orderkey": 579, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5760.36, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-05-03", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ickly final requests-- bold accou" }
+{ "l_orderkey": 992, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13440.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1997-12-29", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the unusual, even dependencies affix fluff" }
+{ "l_orderkey": 1217, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43202.7, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "riously close ideas" }
+{ "l_orderkey": 1540, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33602.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-31", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e blithely a" }
+{ "l_orderkey": 1991, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47042.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-10", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests cajole blithely" }
+{ "l_orderkey": 2086, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21121.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-16", "l_receiptdate": "1994-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "idly busy acc" }
+{ "l_orderkey": 2149, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21121.32, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-24", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ptotes sleep along the blithely ir" }
+{ "l_orderkey": 2213, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3840.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " affix carefully furiously " }
+{ "l_orderkey": 2305, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37442.34, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ms after the foxes " }
+{ "l_orderkey": 2406, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 28801.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final pinto beans han" }
+{ "l_orderkey": 2817, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24001.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-21", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "doze blithely." }
+{ "l_orderkey": 2849, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23041.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e slyly even asymptotes. slo" }
+{ "l_orderkey": 2886, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 960.06, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits fr" }
+{ "l_orderkey": 2944, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16321.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly final dolphins sleep silent the" }
+{ "l_orderkey": 3008, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46082.88, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-23", "l_commitdate": "1996-01-07", "l_receiptdate": "1996-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld theodolites. fluffily bold theodolit" }
+{ "l_orderkey": 3013, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19201.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "unts boost regular ideas. slyly pe" }
+{ "l_orderkey": 3043, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40322.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ide of the un" }
+{ "l_orderkey": 3168, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44162.76, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-02", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the express accounts. fluff" }
+{ "l_orderkey": 4322, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 37442.34, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-16", "l_commitdate": "1998-05-21", "l_receiptdate": "1998-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts. dogged pin" }
+{ "l_orderkey": 4423, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1920.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-04", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old sheaves sleep" }
+{ "l_orderkey": 4743, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18241.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely even accounts" }
+{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4800.3, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ses integrate. regular deposits are about " }
+{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17281.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "blithely ironic theodolites use along" }
+{ "l_orderkey": 4961, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 960.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s affix carefully silent dependen" }
+{ "l_orderkey": 5570, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 27841.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he silent, enticing requests." }
+{ "l_orderkey": 5735, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39362.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-02-10", "l_receiptdate": "1995-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lthily ruthless i" }
+{ "l_orderkey": 33, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30753.92, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-01-04", "l_receiptdate": "1993-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gular theodolites" }
+{ "l_orderkey": 261, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47091.94, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-08", "l_receiptdate": "1993-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pinto beans haggle slyly furiously pending" }
+{ "l_orderkey": 262, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "atelets sleep furiously. requests cajole. b" }
+{ "l_orderkey": 291, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28831.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " fluffily regular deposits. quickl" }
+{ "l_orderkey": 295, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24987.56, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-11-22", "l_receiptdate": "1995-01-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " carefully iron" }
+{ "l_orderkey": 899, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17299.08, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "re daring, pending deposits. blit" }
+{ "l_orderkey": 999, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 32676.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-10-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "its. daringly final instruc" }
+{ "l_orderkey": 1093, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32676.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-09-06", "l_receiptdate": "1997-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sits. express accounts play carefully. bol" }
+{ "l_orderkey": 1319, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20182.26, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s: carefully express " }
+{ "l_orderkey": 1380, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e ironic, even excuses haggle " }
+{ "l_orderkey": 1763, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 42286.64, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " instructions need to integrate deposits. " }
+{ "l_orderkey": 2020, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25948.62, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-14", "l_commitdate": "1993-09-02", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e of the bold foxes haggle " }
+{ "l_orderkey": 2117, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18260.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s between the slyly regula" }
+{ "l_orderkey": 2370, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 30753.92, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ies since the final deposits" }
+{ "l_orderkey": 2374, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1922.12, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", unusual ideas. deposits cajole quietl" }
+{ "l_orderkey": 2595, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40364.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ggle furiou" }
+{ "l_orderkey": 2950, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 44208.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "to the regular accounts are slyly carefu" }
+{ "l_orderkey": 3463, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43247.7, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-04", "l_receiptdate": "1993-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "nts are slyly " }
+{ "l_orderkey": 3682, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5766.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ronic deposits wake slyly. ca" }
+{ "l_orderkey": 3846, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14415.9, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-17", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-02-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uternes. carefully even" }
+{ "l_orderkey": 3968, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6727.42, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-30", "l_commitdate": "1997-05-01", "l_receiptdate": "1997-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully bold instructions. express" }
+{ "l_orderkey": 3974, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16338.02, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-05", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ions eat slyly after the blithely " }
+{ "l_orderkey": 4167, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45169.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-08-24", "l_receiptdate": "1998-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " carefully final asymptotes. slyly bo" }
+{ "l_orderkey": 4672, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 39403.46, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-01", "l_commitdate": "1995-12-15", "l_receiptdate": "1995-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " slyly quie" }
+{ "l_orderkey": 5312, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25948.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-04-09", "l_receiptdate": "1995-04-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tructions cajol" }
+{ "l_orderkey": 5318, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12493.78, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-15", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly silent ideas. ideas haggle among the " }
+{ "l_orderkey": 5376, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40364.52, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y even asymptotes. courts are unusual pa" }
+{ "l_orderkey": 5442, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11532.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-03-18", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully final" }
+{ "l_orderkey": 5797, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16338.02, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ironic, even theodoli" }
+{ "l_orderkey": 33, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-11-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ng to the furiously ironic package" }
+{ "l_orderkey": 71, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24051.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-10", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ckly. slyly" }
+{ "l_orderkey": 102, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14430.9, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "final packages. carefully even excu" }
+{ "l_orderkey": 354, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 34634.16, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "onic requests thrash bold g" }
+{ "l_orderkey": 482, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-01", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithe pin" }
+{ "l_orderkey": 513, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-07-31", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "efully ironic ideas doze slyl" }
+{ "l_orderkey": 1248, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28861.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fily special foxes kindle am" }
+{ "l_orderkey": 1287, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9620.6, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ding, regular accounts" }
+{ "l_orderkey": 1511, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30785.92, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. carefully ironi" }
+{ "l_orderkey": 2087, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 962.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "hely final acc" }
+{ "l_orderkey": 3393, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16355.02, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-15", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly ironic deposits could" }
+{ "l_orderkey": 3907, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21165.32, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly. furiously unusual deposits use afte" }
+{ "l_orderkey": 3936, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11544.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely across the carefully brave req" }
+{ "l_orderkey": 4070, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10582.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-23", "l_commitdate": "1995-08-15", "l_receiptdate": "1995-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully final pack" }
+{ "l_orderkey": 4450, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12506.78, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-09-18", "l_receiptdate": "1997-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " brave foxes. slyly unusual" }
+{ "l_orderkey": 4453, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 46178.88, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-06-24", "l_receiptdate": "1997-06-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "eep. fluffily express accounts at the furi" }
+{ "l_orderkey": 4483, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48103.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ag blithely even" }
+{ "l_orderkey": 5027, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37520.34, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ess requests! quickly regular pac" }
+{ "l_orderkey": 5378, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-17", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans sleep. fu" }
+{ "l_orderkey": 5382, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-26", "l_commitdate": "1992-02-17", "l_receiptdate": "1992-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully regular accounts. slyly ev" }
+{ "l_orderkey": 5536, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "equests mo" }
+{ "l_orderkey": 5602, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rate fluffily regular platelets. blithel" }
+{ "l_orderkey": 5888, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly final accounts hag" }
+{ "l_orderkey": 5955, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14430.9, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-22", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final accounts above the regu" }
+{ "l_orderkey": 3, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 25039.56, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges sleep after the caref" }
+{ "l_orderkey": 100, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26965.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-13", "l_receiptdate": "1998-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sts haggle. slowl" }
+{ "l_orderkey": 869, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-02-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "uffily even excuses? slyly even deposits " }
+{ "l_orderkey": 1636, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-09-09", "l_receiptdate": "1997-08-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular depos" }
+{ "l_orderkey": 1863, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46226.88, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ans hinder furiou" }
+{ "l_orderkey": 1986, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13482.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the packages. pending, unusual" }
+{ "l_orderkey": 2016, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14445.9, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-24", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests haggle carefully furiously regul" }
+{ "l_orderkey": 2624, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14445.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "le. quickly pending requests" }
+{ "l_orderkey": 2791, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3852.24, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-02", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-01-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "slyly bold packages boost. slyly" }
+{ "l_orderkey": 2886, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-18", "l_commitdate": "1995-01-31", "l_receiptdate": "1994-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ar theodolites. e" }
+{ "l_orderkey": 3104, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10593.66, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-05", "l_commitdate": "1993-11-30", "l_receiptdate": "1993-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " special deposits u" }
+{ "l_orderkey": 3264, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5778.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "press packages. ironical" }
+{ "l_orderkey": 3460, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 44300.76, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-02-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uses run among the carefully even deposits" }
+{ "l_orderkey": 3461, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-10", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-03-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ely unusual deposits. quickly ir" }
+{ "l_orderkey": 3618, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23113.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-26", "l_commitdate": "1998-01-15", "l_receiptdate": "1998-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "xpress acc" }
+{ "l_orderkey": 3650, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29854.86, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " against the ironic accounts cajol" }
+{ "l_orderkey": 4130, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uriously regular instructions around th" }
+{ "l_orderkey": 4160, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 46226.88, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " unusual dolphins " }
+{ "l_orderkey": 4451, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " regular ideas." }
+{ "l_orderkey": 4545, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-02-18", "l_receiptdate": "1993-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously bold asymptotes! blithely pen" }
+{ "l_orderkey": 4769, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-26", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ven instructions. ca" }
+{ "l_orderkey": 4871, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2889.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-08-10", "l_receiptdate": "1995-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y special packages wak" }
+{ "l_orderkey": 4998, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45263.82, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-07", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "mong the careful" }
+{ "l_orderkey": 5381, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47189.94, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-06-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " accounts. regular, regula" }
+{ "l_orderkey": 5575, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15408.96, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-10-14", "l_receiptdate": "1995-08-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "jole boldly beyond the final as" }
+{ "l_orderkey": 5702, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29854.86, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-10-22", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "pinto beans. blithely " }
+{ "l_orderkey": 1, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7712.48, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously. regular, express dep" }
+{ "l_orderkey": 352, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16389.02, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending deposits sleep furiously " }
+{ "l_orderkey": 384, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 47238.94, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-04-25", "l_receiptdate": "1992-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully carefully ironic instructions. bl" }
+{ "l_orderkey": 577, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l accounts wake deposits. ironic packa" }
+{ "l_orderkey": 581, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts. quickly" }
+{ "l_orderkey": 1541, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "o beans boost fluffily abou" }
+{ "l_orderkey": 1666, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19281.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-12-12", "l_receiptdate": "1996-01-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uietly regular foxes wake quick" }
+{ "l_orderkey": 2209, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10604.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-12", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-08-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "express, regular pinto be" }
+{ "l_orderkey": 2213, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2892.18, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-03-17", "l_receiptdate": "1993-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "o wake. ironic platel" }
+{ "l_orderkey": 2407, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-10", "l_commitdate": "1998-08-25", "l_receiptdate": "1998-10-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "l dependencies s" }
+{ "l_orderkey": 2470, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9640.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic requests a" }
+{ "l_orderkey": 2628, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40490.52, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-20", "l_commitdate": "1994-01-04", "l_receiptdate": "1993-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ld notornis alongside " }
+{ "l_orderkey": 2755, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20245.26, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-13", "l_commitdate": "1992-04-20", "l_receiptdate": "1992-03-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furious re" }
+{ "l_orderkey": 3172, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". slyly regular dependencies haggle quiet" }
+{ "l_orderkey": 3266, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "grate among the quickly express deposits" }
+{ "l_orderkey": 3271, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27957.74, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-10", "l_commitdate": "1992-02-05", "l_receiptdate": "1992-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lar instructions. carefully regular" }
+{ "l_orderkey": 3331, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "odolites. bold accounts" }
+{ "l_orderkey": 3520, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully pendi" }
+{ "l_orderkey": 3653, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 27957.74, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ording to the special, final" }
+{ "l_orderkey": 3712, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-26", "l_commitdate": "1992-02-19", "l_receiptdate": "1992-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ously permanently regular req" }
+{ "l_orderkey": 3717, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 36634.28, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly about the car" }
+{ "l_orderkey": 4004, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 45310.82, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-08-03", "l_receiptdate": "1993-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "thely instead of the even, unu" }
+{ "l_orderkey": 4037, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30849.92, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e of the pending, iron" }
+{ "l_orderkey": 4039, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "t? pinto beans cajole across the thinly r" }
+{ "l_orderkey": 4545, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1928.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ages use. slyly even i" }
+{ "l_orderkey": 4676, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 12532.78, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-18", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " at the slyly bold attainments. silently e" }
+{ "l_orderkey": 4704, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-02", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the care" }
+{ "l_orderkey": 4832, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5784.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-02-03", "l_receiptdate": "1997-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ages cajole after the bold requests. furi" }
+{ "l_orderkey": 5409, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-26", "l_receiptdate": "1992-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "osits cajole furiously" }
+{ "l_orderkey": 5540, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18317.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1996-11-18", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slyly slyl" }
+{ "l_orderkey": 5826, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17353.08, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-17", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-07-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "atelets use above t" }
+{ "l_orderkey": 70, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7720.48, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ggle. carefully pending dependenc" }
+{ "l_orderkey": 166, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar frays wake blithely a" }
+{ "l_orderkey": 388, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 38602.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1993-01-28", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "quests against the carefully unusual epi" }
+{ "l_orderkey": 419, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30881.92, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely regular requests. special pinto" }
+{ "l_orderkey": 512, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5790.36, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en ideas haggle " }
+{ "l_orderkey": 903, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26056.62, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-09-20", "l_receiptdate": "1995-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lly pending foxes. furiously" }
+{ "l_orderkey": 930, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9650.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckly regular requests: regular instructions" }
+{ "l_orderkey": 935, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22196.38, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-11-25", "l_receiptdate": "1998-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hes haggle furiously dolphins. qu" }
+{ "l_orderkey": 994, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3860.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-05-21", "l_receiptdate": "1994-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "aggle carefully acc" }
+{ "l_orderkey": 1030, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16406.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly. carefully even packages dazz" }
+{ "l_orderkey": 1347, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8685.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-09-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " detect blithely above the fina" }
+{ "l_orderkey": 1409, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34742.16, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ncies sleep carefully r" }
+{ "l_orderkey": 1927, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5790.36, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "furiously even wat" }
+{ "l_orderkey": 2053, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44392.76, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-04-02", "l_receiptdate": "1995-04-18", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tions. furiously even requests hagg" }
+{ "l_orderkey": 2503, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27021.68, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake quickly slyly " }
+{ "l_orderkey": 2528, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-25", "l_commitdate": "1995-02-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng the pending excuses haggle after the bl" }
+{ "l_orderkey": 2563, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9650.6, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tealthily abo" }
+{ "l_orderkey": 2785, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fter the furiously final p" }
+{ "l_orderkey": 2848, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42462.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ions. slyly express instructions n" }
+{ "l_orderkey": 3106, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15440.96, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-03-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "sits wake slyl" }
+{ "l_orderkey": 3270, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19301.2, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en accounts among the c" }
+{ "l_orderkey": 3430, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48253.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-03-03", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic theodolites. carefully regular pac" }
+{ "l_orderkey": 3553, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25091.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fily special p" }
+{ "l_orderkey": 3751, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43427.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "according to " }
+{ "l_orderkey": 4257, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-18", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thin the theodolites use after the bl" }
+{ "l_orderkey": 4354, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-29", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "deas use blithely! special foxes print af" }
+{ "l_orderkey": 4388, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28951.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s cajole fluffil" }
+{ "l_orderkey": 4614, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-08-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ions engage final, ironic " }
+{ "l_orderkey": 4711, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7720.48, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g to the carefully ironic deposits. specia" }
+{ "l_orderkey": 4804, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", thin excuses. " }
+{ "l_orderkey": 4805, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12545.78, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its serve about the accounts. slyly regu" }
+{ "l_orderkey": 4865, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 45357.82, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-08-07", "l_receiptdate": "1997-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y unusual packages. packages" }
+{ "l_orderkey": 4995, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15440.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular, bold packages. accou" }
+{ "l_orderkey": 5095, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44392.76, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular instruction" }
+{ "l_orderkey": 5376, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17371.08, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-09-13", "l_receiptdate": "1994-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts boo" }
+{ "l_orderkey": 5539, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40532.52, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ons across the carefully si" }
+{ "l_orderkey": 71, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2898.18, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-23", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. pinto beans haggle after the" }
+{ "l_orderkey": 194, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12558.78, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-05-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "about the blit" }
+{ "l_orderkey": 261, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ites hinder " }
+{ "l_orderkey": 549, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34778.16, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-10-11", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts against the ironic, even theodolites eng" }
+{ "l_orderkey": 609, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20287.26, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "de of the special warthogs. excu" }
+{ "l_orderkey": 740, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33812.1, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-08-22", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "p quickly. fu" }
+{ "l_orderkey": 995, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-08", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-09-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lyly even " }
+{ "l_orderkey": 1280, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8694.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usual accou" }
+{ "l_orderkey": 1316, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14490.9, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1993-12-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully express dugouts. furiously silent ide" }
+{ "l_orderkey": 1410, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-05-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts haggle against the furiously fina" }
+{ "l_orderkey": 1603, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28015.74, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-10-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ses wake furiously. theodolite" }
+{ "l_orderkey": 1733, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8694.54, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ven foxes was according to t" }
+{ "l_orderkey": 2343, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33812.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ges haggle furiously carefully regular req" }
+{ "l_orderkey": 2562, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts-- silent, unusual ideas a" }
+{ "l_orderkey": 2592, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1932.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-24", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "side of the b" }
+{ "l_orderkey": 2887, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10626.66, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-17", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. unusual, speci" }
+{ "l_orderkey": 2914, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully about the fluffily ironic gifts" }
+{ "l_orderkey": 2950, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uests cajole furio" }
+{ "l_orderkey": 3015, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests wake fluffil" }
+{ "l_orderkey": 3143, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44438.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "low forges haggle. even packages use bli" }
+{ "l_orderkey": 3814, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". doggedly ironic deposits will have to wa" }
+{ "l_orderkey": 4193, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 20287.26, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "accounts cajole b" }
+{ "l_orderkey": 4195, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly express pinto bea" }
+{ "l_orderkey": 4550, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18355.14, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. express " }
+{ "l_orderkey": 4645, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30913.92, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final accounts alongside" }
+{ "l_orderkey": 5380, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5796.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1998-01-08", "l_receiptdate": "1997-12-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. fluffily brave accounts across t" }
+{ "l_orderkey": 5412, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46370.88, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-22", "l_commitdate": "1998-03-28", "l_receiptdate": "1998-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s. slyly final packages cajole blithe" }
+{ "l_orderkey": 5637, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15456.96, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "d packages. express requests" }
+{ "l_orderkey": 5922, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12558.78, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sly special accounts wake ironically." }
+{ "l_orderkey": 612, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 47385.94, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1992-11-25", "l_receiptdate": "1993-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "theodolite" }
+{ "l_orderkey": 1445, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46418.88, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". final ideas are carefully dar" }
+{ "l_orderkey": 1543, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40616.52, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "its sleep until the fur" }
+{ "l_orderkey": 1702, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ies haggle blith" }
+{ "l_orderkey": 1764, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "es wake slowly. " }
+{ "l_orderkey": 2049, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17407.08, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-09", "l_commitdate": "1996-01-22", "l_receiptdate": "1996-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " sleep fluffily. dependencies use never" }
+{ "l_orderkey": 2631, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3868.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-03", "l_commitdate": "1993-12-17", "l_receiptdate": "1993-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "special theodolites. a" }
+{ "l_orderkey": 2661, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10637.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-05-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "equests are a" }
+{ "l_orderkey": 2819, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11604.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-18", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-07-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " regular, regular a" }
+{ "l_orderkey": 3136, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1934.12, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "? special, silent " }
+{ "l_orderkey": 3399, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "hely pending dugouts " }
+{ "l_orderkey": 3426, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "c accounts cajole carefu" }
+{ "l_orderkey": 4102, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37715.34, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ffix blithely slyly special " }
+{ "l_orderkey": 4929, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23209.44, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-04-30", "l_receiptdate": "1996-05-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts boost" }
+{ "l_orderkey": 5317, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48353.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-17", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-11-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole furiously. accounts use quick" }
+{ "l_orderkey": 5344, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25143.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-27", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending, silent multipliers." }
+{ "l_orderkey": 5441, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45451.82, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-16", "l_receiptdate": "1994-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ounts wake slyly about the express instr" }
+{ "l_orderkey": 5507, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21275.32, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gular ideas. carefully unu" }
+{ "l_orderkey": 5543, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-11-05", "l_receiptdate": "1993-12-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress, even " }
+{ "l_orderkey": 5632, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23209.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-23", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "beans detect. quickly final i" }
+{ "l_orderkey": 5734, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9670.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1997-12-24", "l_receiptdate": "1998-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests; accounts above" }
+{ "l_orderkey": 1, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34850.16, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly final dependencies: slyly bold " }
+{ "l_orderkey": 39, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44530.76, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-26", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he carefully e" }
+{ "l_orderkey": 135, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32914.04, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1996-01-19", "l_receiptdate": "1996-02-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "counts doze against the blithely ironi" }
+{ "l_orderkey": 486, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38722.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-06-06", "l_receiptdate": "1996-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts nag quickly among the slyl" }
+{ "l_orderkey": 610, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10648.66, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-10-25", "l_receiptdate": "1995-11-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely final " }
+{ "l_orderkey": 868, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironic platelets wake. rut" }
+{ "l_orderkey": 1473, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30977.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the packages lose furiously ab" }
+{ "l_orderkey": 1507, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24201.5, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-06", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xes. slyly busy de" }
+{ "l_orderkey": 1539, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". fluffily reg" }
+{ "l_orderkey": 1543, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quickly. final accounts haggle slyl" }
+{ "l_orderkey": 1634, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cial, bold platelets alongside of the f" }
+{ "l_orderkey": 1826, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-12", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " blithely special" }
+{ "l_orderkey": 1861, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s foxes. slyly" }
+{ "l_orderkey": 1890, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 41626.58, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly. instructions across the furiously" }
+{ "l_orderkey": 1926, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12584.78, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-26", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "eans wake bli" }
+{ "l_orderkey": 1927, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts affi" }
+{ "l_orderkey": 2052, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48403.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-07-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "wake after the decoy" }
+{ "l_orderkey": 2438, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-09-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "engage car" }
+{ "l_orderkey": 2465, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26137.62, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits boost carefully unusual instructio" }
+{ "l_orderkey": 3041, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "scapades after the special" }
+{ "l_orderkey": 3205, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-17", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly alongsi" }
+{ "l_orderkey": 3842, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12584.78, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-13", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "t blithely. busily regular accounts alon" }
+{ "l_orderkey": 3873, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final ac" }
+{ "l_orderkey": 4706, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5808.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully eve" }
+{ "l_orderkey": 4837, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40658.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "o the furiously final theodolites boost" }
+{ "l_orderkey": 4865, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42594.64, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even deposits sleep against the quickly r" }
+{ "l_orderkey": 5121, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e quickly according " }
+{ "l_orderkey": 5153, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29041.8, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-14", "l_receiptdate": "1995-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "beans sleep bl" }
+{ "l_orderkey": 5252, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23233.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-17", "l_receiptdate": "1996-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "posits after the fluffi" }
+{ "l_orderkey": 5414, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 38722.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts are evenly across" }
+{ "l_orderkey": 5472, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27105.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ffily pendin" }
+{ "l_orderkey": 5504, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3872.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "into beans boost. " }
+{ "l_orderkey": 5602, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29041.8, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e slyly even packages. careful" }
+{ "l_orderkey": 5664, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 32914.04, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-10-05", "l_receiptdate": "1998-09-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "st. fluffily pending foxes na" }
+{ "l_orderkey": 5857, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 968.06, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1997-12-09", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "instructions detect final reques" }
+{ "l_orderkey": 5921, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-05-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "final asymptotes. even packages boost " }
+{ "l_orderkey": 386, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15504.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely fluffi" }
+{ "l_orderkey": 1025, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22288.38, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular platelets nag carefu" }
+{ "l_orderkey": 1221, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2907.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ing to the fluffily" }
+{ "l_orderkey": 1477, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 47483.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-18", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ise according to the sly, bold p" }
+{ "l_orderkey": 1600, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24226.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "press packages. ironic excuses bo" }
+{ "l_orderkey": 1824, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38762.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-24", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es mold furiously final instructions. s" }
+{ "l_orderkey": 2500, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16474.02, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "encies-- ironic, even packages" }
+{ "l_orderkey": 2849, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29071.8, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-07-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly furiously even id" }
+{ "l_orderkey": 3045, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46514.88, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole quickly outside th" }
+{ "l_orderkey": 3205, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 34886.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. ironic platelets above the s" }
+{ "l_orderkey": 3429, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ackages. quickly e" }
+{ "l_orderkey": 3459, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously silent dolphi" }
+{ "l_orderkey": 3591, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23257.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages. slyly regular dependencies cajo" }
+{ "l_orderkey": 3714, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12597.78, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the furiously final" }
+{ "l_orderkey": 3717, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4845.3, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-02", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-09-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "quickly among " }
+{ "l_orderkey": 3940, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38762.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-03-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. regular fox" }
+{ "l_orderkey": 4102, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4845.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-05-11", "l_receiptdate": "1996-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " the furiously even" }
+{ "l_orderkey": 4322, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37793.34, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its integrate fluffily " }
+{ "l_orderkey": 4577, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11628.72, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests alongsi" }
+{ "l_orderkey": 4737, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21319.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " hang fluffily around t" }
+{ "l_orderkey": 4769, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 43607.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-01", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "accounts are. even accounts sleep" }
+{ "l_orderkey": 5216, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16474.02, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s according to the accounts bo" }
+{ "l_orderkey": 5348, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20350.26, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-11", "l_commitdate": "1997-12-24", "l_receiptdate": "1997-12-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular theodolites haggle car" }
+{ "l_orderkey": 5767, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14535.9, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warthogs. carefully unusual g" }
+{ "l_orderkey": 130, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 30072.17, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thily about the ruth" }
+{ "l_orderkey": 417, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "- final requests sle" }
+{ "l_orderkey": 645, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44623.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular dependencies across the speci" }
+{ "l_orderkey": 1155, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3880.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic foxes according to the carefully final " }
+{ "l_orderkey": 1253, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21341.54, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "telets cajole alongside of the final reques" }
+{ "l_orderkey": 1569, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29102.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages. excuses lose evenly carefully reg" }
+{ "l_orderkey": 1984, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33952.45, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "tes. quickly pending packages haggle boldl" }
+{ "l_orderkey": 2213, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 970.07, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s along the ironic reques" }
+{ "l_orderkey": 2629, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32012.31, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-14", "l_receiptdate": "1998-05-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. slowly express accounts are along the" }
+{ "l_orderkey": 2757, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13580.98, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "special deposits u" }
+{ "l_orderkey": 2819, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16491.19, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-16", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "en deposits above the f" }
+{ "l_orderkey": 2949, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48503.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "gular courts cajole across t" }
+{ "l_orderkey": 2977, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24251.75, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-21", "l_commitdate": "1996-10-06", "l_receiptdate": "1996-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "furiously pe" }
+{ "l_orderkey": 2982, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20371.47, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-06-03", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular ideas use furiously? bl" }
+{ "l_orderkey": 3079, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ets are according to the quickly dari" }
+{ "l_orderkey": 3205, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "symptotes. slyly even deposits ar" }
+{ "l_orderkey": 3587, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11640.84, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-07-04", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g the even pinto beans. special," }
+{ "l_orderkey": 3649, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13580.98, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ithely bold accounts wake " }
+{ "l_orderkey": 3650, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 41713.01, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "structions use caref" }
+{ "l_orderkey": 3718, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7760.56, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-06", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " the even deposits sleep carefully b" }
+{ "l_orderkey": 3872, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 40742.94, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-12", "l_receiptdate": "1997-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s the furio" }
+{ "l_orderkey": 3937, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46563.36, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-15", "l_commitdate": "1998-02-22", "l_receiptdate": "1998-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gainst the thinl" }
+{ "l_orderkey": 4197, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ronic requests. quickly bold packages in" }
+{ "l_orderkey": 4199, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15521.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ncies. furiously special accounts" }
+{ "l_orderkey": 4455, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1993-11-21", "l_receiptdate": "1994-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express packages. packages boost quickly" }
+{ "l_orderkey": 4513, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly furiously unusual deposits. blit" }
+{ "l_orderkey": 4992, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24251.75, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-06", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-08-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly about the never ironic requests. pe" }
+{ "l_orderkey": 5059, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4850.35, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-23", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ts affix slyly accordi" }
+{ "l_orderkey": 5124, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34922.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r deposits ab" }
+{ "l_orderkey": 5473, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26191.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the deposits. warthogs wake fur" }
+{ "l_orderkey": 5605, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-13", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cial deposits. theodolites w" }
+{ "l_orderkey": 5636, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-05-17", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "slyly express requests. furiously pen" }
+{ "l_orderkey": 5636, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25221.82, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-05", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously final pinto beans o" }
+{ "l_orderkey": 5984, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12610.91, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-06", "l_receiptdate": "1994-11-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar platelets. f" }
+{ "l_orderkey": 419, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14566.05, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-11-28", "l_receiptdate": "1996-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " sleep final, regular theodolites. fluffi" }
+{ "l_orderkey": 518, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15537.12, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-15", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "use quickly expre" }
+{ "l_orderkey": 547, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42727.08, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-08-17", "l_receiptdate": "1996-10-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely express dependencies. qu" }
+{ "l_orderkey": 641, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24276.75, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "d, regular d" }
+{ "l_orderkey": 673, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21363.54, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-04-27", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " the regular, even requests. carefully fin" }
+{ "l_orderkey": 899, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3884.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-05-14", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. blithe, ironic waters cajole care" }
+{ "l_orderkey": 1543, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33016.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ic requests are ac" }
+{ "l_orderkey": 1606, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13594.98, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "structions haggle f" }
+{ "l_orderkey": 1635, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2913.21, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly ironic r" }
+{ "l_orderkey": 1921, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26218.89, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ing pinto beans above the pend" }
+{ "l_orderkey": 2017, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13594.98, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-07-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ily final w" }
+{ "l_orderkey": 2112, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lphins solve ideas. even, special reque" }
+{ "l_orderkey": 2146, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-10-19", "l_receiptdate": "1993-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y regular foxes wake among the final" }
+{ "l_orderkey": 2183, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28161.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-08-24", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly unusual deposits sleep carefully" }
+{ "l_orderkey": 2212, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-22", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " cajole. final, pending ideas should are bl" }
+{ "l_orderkey": 2341, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35929.59, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-23", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "was blithel" }
+{ "l_orderkey": 2407, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " wake carefully. fluffily " }
+{ "l_orderkey": 2497, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18450.33, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions? carefully daring accounts" }
+{ "l_orderkey": 2884, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39813.87, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1997-12-17", "l_receiptdate": "1998-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep. slyly even accounts a" }
+{ "l_orderkey": 3207, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 40784.94, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-06-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to the quickly special accounts? ironically" }
+{ "l_orderkey": 3360, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 40784.94, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ages cajole. pending, " }
+{ "l_orderkey": 3910, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30103.17, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-22", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess instructions. " }
+{ "l_orderkey": 4295, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3884.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-05", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "arefully according to the pending ac" }
+{ "l_orderkey": 4482, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-16", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-06-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " quickly pendin" }
+{ "l_orderkey": 4544, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19421.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-12", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " waters about the" }
+{ "l_orderkey": 4611, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46611.36, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
+{ "l_orderkey": 4772, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14566.05, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " requests. express, regular th" }
+{ "l_orderkey": 4834, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25247.82, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ages dazzle carefully. slyly daring foxes" }
+{ "l_orderkey": 4839, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8739.63, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-17", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-07-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ounts haggle carefully above" }
+{ "l_orderkey": 4870, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34958.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " instructions. carefully pending pac" }
+{ "l_orderkey": 4967, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48553.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages. final, unusual accounts c" }
+{ "l_orderkey": 5285, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 971.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e fluffily about the slyly special pa" }
+{ "l_orderkey": 5952, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41756.01, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "posits sleep furiously quickly final p" }
+{ "l_orderkey": 486, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " blithely final pinto " }
+{ "l_orderkey": 800, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-09-25", "l_receiptdate": "1998-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "according to the bold, final dependencies " }
+{ "l_orderkey": 1060, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 25273.82, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "quickly abo" }
+{ "l_orderkey": 1185, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7776.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-05", "l_commitdate": "1992-10-05", "l_receiptdate": "1992-12-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely according to the furiously regular r" }
+{ "l_orderkey": 1222, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-12", "l_commitdate": "1993-03-14", "l_receiptdate": "1993-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s print permanently unusual packages. " }
+{ "l_orderkey": 1441, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33050.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e carefully. blithely ironic dep" }
+{ "l_orderkey": 1446, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30134.17, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". slyly reg" }
+{ "l_orderkey": 1477, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30134.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. fluffily final " }
+{ "l_orderkey": 1988, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34994.52, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-21", "l_commitdate": "1995-11-24", "l_receiptdate": "1996-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gular theodolites. " }
+{ "l_orderkey": 2435, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final accounts ar" }
+{ "l_orderkey": 2501, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19441.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. furiou" }
+{ "l_orderkey": 2560, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34994.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts alongside of the excuses are " }
+{ "l_orderkey": 2594, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6804.49, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arls cajole " }
+{ "l_orderkey": 2756, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-05", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ular packages. regular deposi" }
+{ "l_orderkey": 2821, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3888.28, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-19", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ual multipliers. final deposits cajol" }
+{ "l_orderkey": 2945, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular instructions" }
+{ "l_orderkey": 3013, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18469.33, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-18", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fluffily pending packages nag furiously al" }
+{ "l_orderkey": 3077, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24301.75, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lent account" }
+{ "l_orderkey": 3174, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37910.73, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1996-02-08", "l_receiptdate": "1995-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " wake slyly foxes. bold requests p" }
+{ "l_orderkey": 3238, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ackages affix furiously. furiously bol" }
+{ "l_orderkey": 3265, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6804.49, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "he forges. fluffily regular asym" }
+{ "l_orderkey": 3393, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46659.36, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-09-15", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " blithely final reques" }
+{ "l_orderkey": 3558, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16525.19, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely unusual packa" }
+{ "l_orderkey": 3655, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 34022.45, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-17", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng foxes cajole fluffily slyly final fo" }
+{ "l_orderkey": 3969, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 38882.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar requests cajole furiously blithely regu" }
+{ "l_orderkey": 4005, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27217.96, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-01-14", "l_receiptdate": "1996-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y pending dependenc" }
+{ "l_orderkey": 4196, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-07-21", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " accounts. fu" }
+{ "l_orderkey": 4672, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ests. idle, regular ex" }
+{ "l_orderkey": 4742, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14581.05, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-20", "l_commitdate": "1995-05-26", "l_receiptdate": "1995-08-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "terns are sl" }
+{ "l_orderkey": 4806, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5832.42, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "even theodolites. packages sl" }
+{ "l_orderkey": 4898, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42771.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y regular grouches about" }
+{ "l_orderkey": 5313, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-06-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding packages use" }
+{ "l_orderkey": 5443, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37910.73, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-01", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gage carefully across the furiously" }
+{ "l_orderkey": 5540, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23329.68, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits! ironic depths may engage-- b" }
+{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4860.35, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly along the ironic, fi" }
+{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ke slyly against the carefully final req" }
+{ "l_orderkey": 420, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 36003.59, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-12-13", "l_receiptdate": "1995-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "rbits. bold requests along the quickl" }
+{ "l_orderkey": 676, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-05", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ress, regular dep" }
+{ "l_orderkey": 998, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-05", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-01-13", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "es sleep. regular dependencies use bl" }
+{ "l_orderkey": 1378, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10703.77, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely express hoc" }
+{ "l_orderkey": 1379, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12649.91, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-08", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-06-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ully across the furiously iron" }
+{ "l_orderkey": 1637, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly final pinto beans. furiously" }
+{ "l_orderkey": 1732, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-02", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ix carefully at the furiously regular pac" }
+{ "l_orderkey": 1761, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 47680.43, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y even packages promise" }
+{ "l_orderkey": 1762, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-11-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely brave" }
+{ "l_orderkey": 1924, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6811.49, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "osits. even accounts nag furious" }
+{ "l_orderkey": 1927, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14596.05, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " carefully regular requests sleep car" }
+{ "l_orderkey": 1958, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8757.63, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly. slyly bold " }
+{ "l_orderkey": 2050, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tside the blithely pending packages eat f" }
+{ "l_orderkey": 2215, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32111.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites cajole b" }
+{ "l_orderkey": 2757, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16542.19, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "er the furiously silent " }
+{ "l_orderkey": 2790, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 12649.91, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "n deposits according to the regul" }
+{ "l_orderkey": 2951, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20434.47, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nt instructions toward the f" }
+{ "l_orderkey": 3783, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35030.52, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular accounts" }
+{ "l_orderkey": 3840, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43788.15, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-12", "l_commitdate": "1998-10-12", "l_receiptdate": "1998-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "onic, even packages are. pe" }
+{ "l_orderkey": 4167, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-11", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "xpress platelets. blithely " }
+{ "l_orderkey": 4614, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-01", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "regular, even" }
+{ "l_orderkey": 4741, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas boost furiously slyly regular id" }
+{ "l_orderkey": 4743, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20434.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake blithely against the packages. reg" }
+{ "l_orderkey": 4868, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle unusual, fluffy packages. foxes cajol" }
+{ "l_orderkey": 4994, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5838.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "grate carefully around th" }
+{ "l_orderkey": 5601, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 36976.66, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-08", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ter the evenly final deposit" }
+{ "l_orderkey": 5824, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-01-07", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly express Ti" }
+{ "l_orderkey": 65, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21429.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ideas. special, r" }
+{ "l_orderkey": 801, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-22", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " even asymptotes" }
+{ "l_orderkey": 929, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13636.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-11-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "gainst the" }
+{ "l_orderkey": 1156, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47729.43, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithely unusual in" }
+{ "l_orderkey": 1888, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 37014.66, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1994-01-16", "l_receiptdate": "1993-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dazzle carefull" }
+{ "l_orderkey": 2208, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 39936.87, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-06-19", "l_receiptdate": "1995-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nd the furious, express dependencies." }
+{ "l_orderkey": 2245, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-19", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e requests sleep furiou" }
+{ "l_orderkey": 2374, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ets cajole fu" }
+{ "l_orderkey": 2528, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12662.91, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1995-01-20", "l_receiptdate": "1994-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ggle furiously. slyly final asympt" }
+{ "l_orderkey": 3046, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42859.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " are quickly. blithe" }
+{ "l_orderkey": 3460, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2922.21, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "er quickly " }
+{ "l_orderkey": 3587, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 22403.61, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l multipliers sleep theodolites-- slyly " }
+{ "l_orderkey": 4004, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ncies. slyly pending dolphins sleep furio" }
+{ "l_orderkey": 4097, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48703.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-31", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. blithely pending" }
+{ "l_orderkey": 4097, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-29", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " even depend" }
+{ "l_orderkey": 4100, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3896.28, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-04-29", "l_receiptdate": "1996-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly regular, bold requ" }
+{ "l_orderkey": 4162, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-21", "l_commitdate": "1992-05-02", "l_receiptdate": "1992-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "elets. slyly regular i" }
+{ "l_orderkey": 4262, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-09-09", "l_receiptdate": "1996-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages boost. pending, even instruction" }
+{ "l_orderkey": 4288, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31170.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-19", "l_commitdate": "1993-01-26", "l_receiptdate": "1993-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e blithely even instructions. speci" }
+{ "l_orderkey": 4578, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9740.7, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1992-11-19", "l_receiptdate": "1993-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uests. blithely unus" }
+{ "l_orderkey": 4775, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 974.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "furiously ironic theodolite" }
+{ "l_orderkey": 5093, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14611.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-02", "l_commitdate": "1993-11-18", "l_receiptdate": "1994-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly among the unusual foxe" }
+{ "l_orderkey": 5412, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30196.17, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-23", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "t the accounts detect slyly about the c" }
+{ "l_orderkey": 5606, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ow requests wake around the regular accoun" }
+{ "l_orderkey": 5831, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32144.31, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions wake. slyly sil" }
+{ "l_orderkey": 420, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11700.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c instructions are " }
+{ "l_orderkey": 581, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29252.1, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-05-21", "l_receiptdate": "1997-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " regular ideas grow furio" }
+{ "l_orderkey": 900, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23401.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-21", "l_commitdate": "1994-12-25", "l_receiptdate": "1994-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "-ray furiously un" }
+{ "l_orderkey": 992, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 39977.87, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-14", "l_commitdate": "1998-02-04", "l_receiptdate": "1997-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eodolites cajole across the accounts." }
+{ "l_orderkey": 1124, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32177.31, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-19", "l_commitdate": "1998-09-17", "l_receiptdate": "1998-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "eposits sleep slyly. stealthily f" }
+{ "l_orderkey": 1216, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46803.36, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-17", "l_commitdate": "1993-02-01", "l_receiptdate": "1993-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "symptotes use against th" }
+{ "l_orderkey": 1569, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-16", "l_commitdate": "1998-06-21", "l_receiptdate": "1998-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages. ironic, even excuses a" }
+{ "l_orderkey": 1668, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40952.94, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ole carefully excuses. final" }
+{ "l_orderkey": 1697, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5850.42, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-11-27", "l_receiptdate": "1997-01-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "accounts breach slyly even de" }
+{ "l_orderkey": 1859, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17551.26, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e carefully a" }
+{ "l_orderkey": 2082, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35102.52, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "haggle furiously silent pinto beans" }
+{ "l_orderkey": 2534, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 41928.01, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-25", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ngly final depos" }
+{ "l_orderkey": 2791, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8775.63, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "pendencies. blithely bold patterns acr" }
+{ "l_orderkey": 2944, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17551.26, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-07", "l_commitdate": "1997-10-26", "l_receiptdate": "1998-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " furiously slyl" }
+{ "l_orderkey": 3014, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "es are. final braids nag slyly. fluff" }
+{ "l_orderkey": 3234, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15601.12, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ithely ironic accounts wake along t" }
+{ "l_orderkey": 3396, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31202.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-07", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits are slyly. final, bold foxes s" }
+{ "l_orderkey": 3494, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22426.61, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits nag " }
+{ "l_orderkey": 3525, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30227.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-27", "l_receiptdate": "1996-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he careful" }
+{ "l_orderkey": 4000, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 42903.08, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-27", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "equests use blithely blithely bold d" }
+{ "l_orderkey": 4103, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usly across the slyly busy accounts! fin" }
+{ "l_orderkey": 4227, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10725.77, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-02", "l_receiptdate": "1995-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l requests-- bold requests cajole dogg" }
+{ "l_orderkey": 4230, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-12", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-06-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nt instruct" }
+{ "l_orderkey": 4417, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-09-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ies across the furious" }
+{ "l_orderkey": 4610, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25351.82, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-07-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " to the fluffily ironic requests h" }
+{ "l_orderkey": 4708, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely. carefully sp" }
+{ "l_orderkey": 5062, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3900.28, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-14", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ke furiously express theodolites. " }
+{ "l_orderkey": 5472, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-05-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e requests detect furiously. ruthlessly un" }
+{ "l_orderkey": 5892, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22426.61, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " foxes nag slyly about the qui" }
+{ "l_orderkey": 486, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35138.52, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "deposits around the quickly regular packa" }
+{ "l_orderkey": 676, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 32210.31, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "as wake slyly furiously close pinto b" }
+{ "l_orderkey": 804, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42947.08, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly final deposits? special " }
+{ "l_orderkey": 805, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25377.82, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-09-24", "l_receiptdate": "1995-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". ironic deposits sleep across " }
+{ "l_orderkey": 1120, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20497.47, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s: fluffily even packages c" }
+{ "l_orderkey": 1408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10736.77, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y even accounts thrash care" }
+{ "l_orderkey": 1607, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33186.38, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-06", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-01-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly above the " }
+{ "l_orderkey": 1634, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1952.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. carefully regular asymptotes wake" }
+{ "l_orderkey": 1924, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14641.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1996-11-13", "l_receiptdate": "1997-01-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he package" }
+{ "l_orderkey": 2214, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26353.89, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "x fluffily along the even packages-- " }
+{ "l_orderkey": 2245, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42947.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully even sheaves" }
+{ "l_orderkey": 2272, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11712.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-04-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " accounts cajole. quickly b" }
+{ "l_orderkey": 2565, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25377.82, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ites wake. ironic acco" }
+{ "l_orderkey": 3649, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39042.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffy somas sleep quickly-- ironic de" }
+{ "l_orderkey": 3751, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38066.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to beans. pending, express packages c" }
+{ "l_orderkey": 4066, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7808.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-05-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts. special pinto beans" }
+{ "l_orderkey": 4262, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 29282.1, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "tes after the carefully" }
+{ "l_orderkey": 4836, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13664.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites. unusual, bold dolphins ar" }
+{ "l_orderkey": 4963, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15617.12, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " carefully slyly u" }
+{ "l_orderkey": 4966, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9760.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. carefully pending requests" }
+{ "l_orderkey": 5408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33186.38, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "requests detect blithely a" }
+{ "l_orderkey": 451, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27357.96, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " theodolites. even cou" }
+{ "l_orderkey": 484, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 46899.36, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-03-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, bold packages? even mult" }
+{ "l_orderkey": 1157, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44945.22, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "slyly regular excuses. accounts" }
+{ "l_orderkey": 1411, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29312.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ious foxes wake courts. caref" }
+{ "l_orderkey": 1635, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39082.8, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uriously up the ironic deposits. slyly i" }
+{ "l_orderkey": 1891, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-20", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ests along" }
+{ "l_orderkey": 2497, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14656.05, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1992-11-20", "l_receiptdate": "1993-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sly against the" }
+{ "l_orderkey": 3494, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29312.1, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-07-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ns are quickly regular, " }
+{ "l_orderkey": 3815, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2931.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular, express ideas. ironic, final dep" }
+{ "l_orderkey": 4166, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4885.35, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-06-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "hely unusual packages are above the f" }
+{ "l_orderkey": 4293, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48853.5, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " special deposits. furiousl" }
+{ "l_orderkey": 4546, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3908.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly pending dependencies along the furio" }
+{ "l_orderkey": 4708, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31266.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the accounts. e" }
+{ "l_orderkey": 4900, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32243.31, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "nto beans nag slyly reg" }
+{ "l_orderkey": 4929, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31266.24, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly at the blithely pending pl" }
+{ "l_orderkey": 5059, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "enly. requests doze. express, close pa" }
+{ "l_orderkey": 5477, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20518.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-28", "l_commitdate": "1998-02-15", "l_receiptdate": "1998-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "blate slyly. silent" }
+{ "l_orderkey": 5702, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42991.08, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-11-25", "l_receiptdate": "1994-01-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lites. carefully final requests doze b" }
+{ "l_orderkey": 5824, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39082.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "he final packag" }
+{ "l_orderkey": 5889, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16610.19, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "blithely pending packages. flu" }
+{ "l_orderkey": 97, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18583.33, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gifts. furiously ironic packages cajole. " }
+{ "l_orderkey": 129, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 21517.54, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e. fluffily regular " }
+{ "l_orderkey": 353, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44991.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ironic dolphins " }
+{ "l_orderkey": 676, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19561.4, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-01", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "riously around the blithely " }
+{ "l_orderkey": 771, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-08-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "packages affix slyly about the quickly " }
+{ "l_orderkey": 1187, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39122.8, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-03-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar, brave deposits nag blithe" }
+{ "l_orderkey": 1251, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-07", "l_receiptdate": "1997-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic Tiresias are slyly furio" }
+{ "l_orderkey": 1281, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 42057.01, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-28", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "final accounts. final packages slee" }
+{ "l_orderkey": 1380, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14671.05, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-08-12", "l_receiptdate": "1996-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "riously ironic foxes aff" }
+{ "l_orderkey": 1665, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly final p" }
+{ "l_orderkey": 1764, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly final foxes wake blithely even requests" }
+{ "l_orderkey": 2022, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-04", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " orbits haggle fluffily fl" }
+{ "l_orderkey": 2145, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "alongside of the slyly final" }
+{ "l_orderkey": 2150, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25429.82, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-21", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-06-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". always unusual packages" }
+{ "l_orderkey": 2178, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2934.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-01-23", "l_receiptdate": "1997-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " permanentl" }
+{ "l_orderkey": 2210, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " requests wake enticingly final" }
+{ "l_orderkey": 2240, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 23473.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ng the silent accounts. slyly ironic t" }
+{ "l_orderkey": 2532, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48903.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1996-01-01", "l_receiptdate": "1995-11-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly after the fluffily regul" }
+{ "l_orderkey": 2657, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41078.94, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-23", "l_commitdate": "1995-11-22", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ckly slyly even accounts. platelets x-ray" }
+{ "l_orderkey": 2882, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-04", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "rding to the regu" }
+{ "l_orderkey": 2918, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23473.68, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-20", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " quickly. express requests haggle careful" }
+{ "l_orderkey": 3077, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "luffily close depende" }
+{ "l_orderkey": 3078, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20539.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e fluffily. " }
+{ "l_orderkey": 3719, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18583.33, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he regular ideas integrate acros" }
+{ "l_orderkey": 4359, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-05-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " fluffily ironic, bold pac" }
+{ "l_orderkey": 4514, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake furiously. carefully regular requests" }
+{ "l_orderkey": 4704, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13692.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-27", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " above the slyly final requests. quickly " }
+{ "l_orderkey": 5025, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly silent deposits boost busily again" }
+{ "l_orderkey": 5088, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22495.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-03", "l_commitdate": "1993-03-07", "l_receiptdate": "1993-03-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "cording to the fluffily expr" }
+{ "l_orderkey": 5091, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48903.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al dependencies. r" }
+{ "l_orderkey": 5126, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22495.61, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular, blithe packages." }
+{ "l_orderkey": 5538, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8802.63, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "encies across the blithely fina" }
+{ "l_orderkey": 5604, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-06-23", "l_receiptdate": "1998-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly final realms wake blit" }
+{ "l_orderkey": 5829, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-31", "l_receiptdate": "1997-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ns about the excuses are c" }
+{ "l_orderkey": 5895, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14671.05, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "silent package" }
+{ "l_orderkey": 229, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27413.96, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final, regular requests. platel" }
+{ "l_orderkey": 288, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18602.33, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-03-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "deposits. blithely quick courts ar" }
+{ "l_orderkey": 450, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1958.14, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-05-21", "l_receiptdate": "1995-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "y even pinto beans; qui" }
+{ "l_orderkey": 482, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18602.33, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-04-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts hinder carefully silent requests" }
+{ "l_orderkey": 514, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20560.47, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-09", "l_commitdate": "1996-05-15", "l_receiptdate": "1996-07-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s sleep quickly blithely" }
+{ "l_orderkey": 613, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5874.42, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-09", "l_receiptdate": "1995-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic deposits eat " }
+{ "l_orderkey": 1252, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "onic pinto beans haggle furiously " }
+{ "l_orderkey": 1316, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14686.05, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-12", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "se. furiously final depo" }
+{ "l_orderkey": 1669, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23497.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " regular, final deposits use quick" }
+{ "l_orderkey": 1828, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13706.98, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final packages along the carefully bold" }
+{ "l_orderkey": 1957, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48953.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "gainst the re" }
+{ "l_orderkey": 1988, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-12-15", "l_receiptdate": "1996-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic dolphins haggl" }
+{ "l_orderkey": 2211, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-21", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly final" }
+{ "l_orderkey": 2466, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29372.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". fluffily even pinto beans are idly. f" }
+{ "l_orderkey": 2657, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24476.75, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly pinto beans. final " }
+{ "l_orderkey": 2690, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 34267.45, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y silent pinto be" }
+{ "l_orderkey": 3109, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " sleep slyly according to t" }
+{ "l_orderkey": 3141, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8811.63, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly ironic, pendi" }
+{ "l_orderkey": 3234, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44058.15, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-15", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " express packages are carefully. f" }
+{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "as sleep carefully into the caref" }
+{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-25", "l_receiptdate": "1996-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uctions wake fluffily. care" }
+{ "l_orderkey": 3555, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14686.05, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y across the pending a" }
+{ "l_orderkey": 3969, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 45037.22, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "fully final requests sleep stealthily. care" }
+{ "l_orderkey": 4038, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23497.68, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-01", "l_commitdate": "1996-04-05", "l_receiptdate": "1996-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ake quickly after the final, ironic ac" }
+{ "l_orderkey": 4069, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21539.54, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-08-04", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts. slyly special instruction" }
+{ "l_orderkey": 4293, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 44058.15, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar ideas use carefully" }
+{ "l_orderkey": 4389, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38183.73, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual, final excuses cajole carefully " }
+{ "l_orderkey": 4418, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-05-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "luffily across the unusual ideas. reque" }
+{ "l_orderkey": 4608, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 48953.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " wake closely. even decoys haggle above" }
+{ "l_orderkey": 4929, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unts against " }
+{ "l_orderkey": 4997, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43079.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-07-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r escapades ca" }
+{ "l_orderkey": 5094, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20560.47, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-26", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-08-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithely furiously final re" }
+{ "l_orderkey": 5155, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38183.73, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-09-01", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l dolphins nag caref" }
+{ "l_orderkey": 5344, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-07-26", "l_receiptdate": "1998-11-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "thely express packages" }
+{ "l_orderkey": 5377, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-21", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely ironic theodolites are care" }
+{ "l_orderkey": 5894, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-09-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " asymptotes among the blithely silent " }
+{ "l_orderkey": 5986, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e fluffily ironic ideas. silent " }
+{ "l_orderkey": 7, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34302.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-16", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "jole. excuses wake carefully alongside of " }
+{ "l_orderkey": 483, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22541.84, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-20", "l_commitdate": "1995-08-11", "l_receiptdate": "1995-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "requests was quickly against th" }
+{ "l_orderkey": 644, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6860.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular requests are blithely. slyly" }
+{ "l_orderkey": 736, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22541.84, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-08", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "k accounts are carefully" }
+{ "l_orderkey": 807, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9800.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-01-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously final depths sleep a" }
+{ "l_orderkey": 864, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33322.72, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously ironic platelets! " }
+{ "l_orderkey": 962, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2940.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ag furiously. even pa" }
+{ "l_orderkey": 1121, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 37.0, "l_extendedprice": 36262.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-04", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "special packages. fluffily final requests s" }
+{ "l_orderkey": 1827, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23521.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al gifts! re" }
+{ "l_orderkey": 1888, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 48023.92, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-01-14", "l_receiptdate": "1994-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar accounts haggle carefu" }
+{ "l_orderkey": 2245, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32342.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-07-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across the express reques" }
+{ "l_orderkey": 2500, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40183.28, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s could have to integrate after the " }
+{ "l_orderkey": 2756, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46063.76, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-05-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e final, f" }
+{ "l_orderkey": 3136, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 28422.32, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-16", "l_commitdate": "1994-10-03", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "latelets. final " }
+{ "l_orderkey": 3623, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31362.56, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-03-15", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " courts. furiously regular ideas b" }
+{ "l_orderkey": 3652, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 980.08, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " bold dependencies sublate. r" }
+{ "l_orderkey": 3750, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-06-06", "l_receiptdate": "1995-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss, ironic requests! fur" }
+{ "l_orderkey": 3877, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43123.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-07-15", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "elets. quickly regular accounts caj" }
+{ "l_orderkey": 4295, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29402.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "yly ironic frets. pending foxes after " }
+{ "l_orderkey": 4422, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-07-16", "l_receiptdate": "1995-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ructions wake slyly al" }
+{ "l_orderkey": 4868, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33322.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-06-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "en instructions about th" }
+{ "l_orderkey": 4994, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46063.76, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sts. blithely close ideas sleep quic" }
+{ "l_orderkey": 5030, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss excuses serve bli" }
+{ "l_orderkey": 5090, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29402.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-04", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "osits nag slyly. fluffily ex" }
+{ "l_orderkey": 5184, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 48023.92, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thlessly closely even reque" }
+{ "l_orderkey": 5217, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. express, express accounts c" }
+{ "l_orderkey": 5346, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 40183.28, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-10", "l_commitdate": "1994-02-15", "l_receiptdate": "1994-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "fully close instructi" }
+{ "l_orderkey": 5477, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-02-09", "l_receiptdate": "1998-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "platelets about the ironic" }
+{ "l_orderkey": 5543, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 31362.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ully around the " }
+{ "l_orderkey": 611, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 981.08, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-02-26", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ts. pending platelets aff" }
+{ "l_orderkey": 1221, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y slyly above the slyly unusual ideas" }
+{ "l_orderkey": 1312, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8829.72, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-07-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". furiously " }
+{ "l_orderkey": 1347, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44148.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-09-03", "l_receiptdate": "1997-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ages wake around t" }
+{ "l_orderkey": 2177, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22564.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he silent foxes. iro" }
+{ "l_orderkey": 2240, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-11", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-04-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss thinly deposits. blithely bold package" }
+{ "l_orderkey": 2567, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45129.68, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "efully pending epitaphs. carefully reg" }
+{ "l_orderkey": 2915, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-24", "l_receiptdate": "1994-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "into beans dazzle alongside of" }
+{ "l_orderkey": 3238, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 981.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-05-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "wake alongs" }
+{ "l_orderkey": 3364, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2943.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c theodolites. blithely ir" }
+{ "l_orderkey": 3394, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13735.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e furiously final theodolites. furio" }
+{ "l_orderkey": 3430, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-01-28", "l_receiptdate": "1995-02-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "egular instruction" }
+{ "l_orderkey": 3687, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1962.16, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-03-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " express requests. slyly regular depend" }
+{ "l_orderkey": 3875, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23545.92, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ecial packages. " }
+{ "l_orderkey": 4514, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8829.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "wly. quick" }
+{ "l_orderkey": 4675, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17659.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-03-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cajole unusual dep" }
+{ "l_orderkey": 4995, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-03-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts. blithely silent ideas after t" }
+{ "l_orderkey": 5027, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49054.0, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " beans dazzle according to the fluffi" }
+{ "l_orderkey": 5217, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46110.76, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-24", "l_commitdate": "1995-12-25", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic packages i" }
+{ "l_orderkey": 5248, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38262.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-04", "l_receiptdate": "1995-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yly even accounts. spe" }
+{ "l_orderkey": 5954, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39243.2, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-30", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "iously ironic deposits after" }
+{ "l_orderkey": 481, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10802.88, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-12", "l_commitdate": "1992-11-17", "l_receiptdate": "1993-02-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "eful attai" }
+{ "l_orderkey": 771, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 22587.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "cajole besides the quickly ironic pin" }
+{ "l_orderkey": 931, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37319.04, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-02-24", "l_receiptdate": "1993-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly final packages integrate carefully" }
+{ "l_orderkey": 1444, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly among the bol" }
+{ "l_orderkey": 1504, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41247.36, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep. carefully ironic excuses haggle quickl" }
+{ "l_orderkey": 1830, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35354.88, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-21", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slowly unusual orbits. carefull" }
+{ "l_orderkey": 2241, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-16", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-08-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ", express deposits. pear" }
+{ "l_orderkey": 2468, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4910.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-28", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-07-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " sleep fluffily acc" }
+{ "l_orderkey": 2595, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 30444.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-02-10", "l_receiptdate": "1996-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tipliers w" }
+{ "l_orderkey": 2723, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-24", "l_commitdate": "1995-11-15", "l_receiptdate": "1996-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bold foxes are bold packages. regular, fin" }
+{ "l_orderkey": 2976, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21605.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ncies kindle furiously. carefull" }
+{ "l_orderkey": 3360, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38301.12, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. blithely express pinto bean" }
+{ "l_orderkey": 4261, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3928.32, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-11", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ackages unwind furiously fluff" }
+{ "l_orderkey": 4324, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13749.12, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages nag express excuses. qui" }
+{ "l_orderkey": 4387, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2946.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pinto beans " }
+{ "l_orderkey": 4611, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49104.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "l platelets. " }
+{ "l_orderkey": 4867, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6874.56, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-08-17", "l_receiptdate": "1992-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e carefully even packages. slyly ironic i" }
+{ "l_orderkey": 4933, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ctions nag final instructions. accou" }
+{ "l_orderkey": 5122, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42229.44, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-31", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the carefully special foxes. idle," }
+{ "l_orderkey": 5186, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34372.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sly silent pack" }
+{ "l_orderkey": 5317, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28480.32, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-28", "l_commitdate": "1994-11-27", "l_receiptdate": "1994-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "oss the carefull" }
+{ "l_orderkey": 5606, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29462.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " nag always. blithely express packages " }
+{ "l_orderkey": 5831, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5892.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts nag pendin" }
+{ "l_orderkey": 32, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27526.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-23", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sleep quickly. req" }
+{ "l_orderkey": 68, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 26543.16, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ccounts. deposits use. furiously" }
+{ "l_orderkey": 192, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24577.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": ". carefully regular" }
+{ "l_orderkey": 226, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 47187.84, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-11", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "efully silent packages. final deposit" }
+{ "l_orderkey": 358, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 44238.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-08", "l_commitdate": "1993-10-29", "l_receiptdate": "1993-12-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans. regular, unusual deposits sl" }
+{ "l_orderkey": 487, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1966.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oss the unusual pinto beans. reg" }
+{ "l_orderkey": 512, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xes. pinto beans cajole carefully; " }
+{ "l_orderkey": 1120, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-10", "l_commitdate": "1998-02-01", "l_receiptdate": "1997-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ages haggle furiously " }
+{ "l_orderkey": 1157, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3932.32, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-30", "l_receiptdate": "1998-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ounts. ironic deposits" }
+{ "l_orderkey": 1411, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45221.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly daring instructions" }
+{ "l_orderkey": 1573, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15729.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-15", "l_commitdate": "1993-03-16", "l_receiptdate": "1993-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ely. furiously final requests wake slyl" }
+{ "l_orderkey": 1575, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "k excuses. pinto beans wake a" }
+{ "l_orderkey": 1958, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37357.04, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-09", "l_commitdate": "1995-11-26", "l_receiptdate": "1995-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "yly. slyly regular courts use silentl" }
+{ "l_orderkey": 2403, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33424.72, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-19", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly bold re" }
+{ "l_orderkey": 2916, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20644.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-11", "l_commitdate": "1996-02-21", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express ideas over the slyly even " }
+{ "l_orderkey": 3040, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13763.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle carefully. express hocke" }
+{ "l_orderkey": 3072, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38340.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es; slyly spe" }
+{ "l_orderkey": 3207, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7864.64, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-13", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. final pint" }
+{ "l_orderkey": 3269, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 36373.96, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-14", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "egular requests. carefully un" }
+{ "l_orderkey": 3750, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-06-25", "l_receiptdate": "1995-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "l dolphins against the slyly" }
+{ "l_orderkey": 3814, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46204.76, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual requests. bli" }
+{ "l_orderkey": 3936, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41289.36, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "elets wake amo" }
+{ "l_orderkey": 4067, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 16712.36, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts affix. regular, regular requests s" }
+{ "l_orderkey": 4738, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 27526.24, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-06-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e furiously ironic excuses. care" }
+{ "l_orderkey": 4902, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-12", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-11-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "daring foxes? even, bold requests wake f" }
+{ "l_orderkey": 5187, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "aggle never bold " }
+{ "l_orderkey": 5218, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-04", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-08-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "k theodolites. express, even id" }
+{ "l_orderkey": 5220, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26543.16, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-29", "l_receiptdate": "1992-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole blithely furiously iron" }
+{ "l_orderkey": 5345, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2949.24, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-10-03", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ites wake carefully unusual " }
+{ "l_orderkey": 5347, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47187.84, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-03-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "equests are slyly. blithely regu" }
+{ "l_orderkey": 5443, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39323.2, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "n courts. special re" }
+{ "l_orderkey": 5635, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "cross the d" }
+{ "l_orderkey": 5765, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 40306.28, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " furiously. slyly sile" }
+{ "l_orderkey": 229, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19681.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "le. instructions use across the quickly fin" }
+{ "l_orderkey": 231, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45267.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix blithely. bold requests among the f" }
+{ "l_orderkey": 518, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-20", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " special requests. fluffily ironic re" }
+{ "l_orderkey": 772, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9840.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-05-19", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " deposits cajole carefully instructions. t" }
+{ "l_orderkey": 836, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17713.44, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y pending packages use alon" }
+{ "l_orderkey": 1190, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31490.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y final packages? slyly even" }
+{ "l_orderkey": 1285, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32474.64, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-08", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-09-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ites affix" }
+{ "l_orderkey": 1377, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25586.08, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular deposits. quickly regular acco" }
+{ "l_orderkey": 1510, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly brave theod" }
+{ "l_orderkey": 1920, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e blithely unusual foxes. brave packages" }
+{ "l_orderkey": 2017, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10824.88, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "gside of the slyly dogged dolp" }
+{ "l_orderkey": 2049, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16729.36, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-04", "l_commitdate": "1996-03-01", "l_receiptdate": "1996-02-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "al, regular foxes. pending, " }
+{ "l_orderkey": 2086, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26570.16, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-04", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "theodolites haggle blithely blithe p" }
+{ "l_orderkey": 2471, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36410.96, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts mold blithely carefully express depo" }
+{ "l_orderkey": 2501, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3936.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-07-27", "l_receiptdate": "1997-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests. furiously final" }
+{ "l_orderkey": 2597, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-15", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending packages. enticingly fi" }
+{ "l_orderkey": 3170, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 25586.08, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s engage furiously. " }
+{ "l_orderkey": 3234, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-15", "l_receiptdate": "1996-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d-- fluffily special packag" }
+{ "l_orderkey": 3296, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11808.96, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "y about the slyly bold pinto bea" }
+{ "l_orderkey": 3332, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27554.24, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the carefully special multipl" }
+{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28538.32, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly unusual i" }
+{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32474.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully across the fur" }
+{ "l_orderkey": 3936, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 34442.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly ironic requ" }
+{ "l_orderkey": 4324, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21649.76, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-13", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ke express, special ideas." }
+{ "l_orderkey": 4357, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-12-03", "l_receiptdate": "1997-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s. final, e" }
+{ "l_orderkey": 4388, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27554.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ove the ide" }
+{ "l_orderkey": 4774, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44283.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " haggle busily afte" }
+{ "l_orderkey": 5895, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 48219.92, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "permanent foxes. packages" }
+{ "l_orderkey": 195, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5910.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-09", "l_commitdate": "1994-03-27", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y, even deposits haggle carefully. bli" }
+{ "l_orderkey": 263, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8865.72, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-07-16", "l_receiptdate": "1994-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lms wake bl" }
+{ "l_orderkey": 326, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4925.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas sleep according to the sometimes spe" }
+{ "l_orderkey": 390, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 23641.92, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-18", "l_commitdate": "1998-05-19", "l_receiptdate": "1998-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. enticingly final depos" }
+{ "l_orderkey": 546, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15761.28, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1996-12-30", "l_receiptdate": "1997-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "de of the orbits. sometimes regula" }
+{ "l_orderkey": 580, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32507.64, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-11", "l_commitdate": "1997-09-19", "l_receiptdate": "1997-10-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y express theodolites cajole carefully " }
+{ "l_orderkey": 644, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 32507.64, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-26", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-08-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ages sleep. bold, bo" }
+{ "l_orderkey": 739, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27582.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets about the pe" }
+{ "l_orderkey": 800, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-10-01", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ckly even requests after the carefully r" }
+{ "l_orderkey": 899, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3940.32, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-06-28", "l_receiptdate": "1998-06-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ter the carefully regular deposits are agai" }
+{ "l_orderkey": 967, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3940.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "platelets hang carefully along " }
+{ "l_orderkey": 1057, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-28", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-03-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ar orbits boost bli" }
+{ "l_orderkey": 1221, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-27", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress accounts " }
+{ "l_orderkey": 1636, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1970.16, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-10-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nal foxes cajole above the blithely reg" }
+{ "l_orderkey": 1794, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33492.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rs above the accoun" }
+{ "l_orderkey": 2021, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts boost blithely. blithely reg" }
+{ "l_orderkey": 2081, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-06", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ual requests wake blithely above the" }
+{ "l_orderkey": 2211, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-05", "l_commitdate": "1994-09-13", "l_receiptdate": "1994-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ependencies " }
+{ "l_orderkey": 2273, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 34477.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-01-19", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "arefully f" }
+{ "l_orderkey": 3076, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43343.52, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-14", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " instructions h" }
+{ "l_orderkey": 3175, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13791.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-21", "l_commitdate": "1994-09-05", "l_receiptdate": "1994-11-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nt dependencies are quietly even " }
+{ "l_orderkey": 3783, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49254.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he furiously regular deposits. " }
+{ "l_orderkey": 4032, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9850.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-31", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully bol" }
+{ "l_orderkey": 4067, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 16746.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r accounts. slyly special pa" }
+{ "l_orderkey": 4224, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 47283.84, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-03", "l_commitdate": "1997-08-31", "l_receiptdate": "1997-10-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " final, regular asymptotes use alway" }
+{ "l_orderkey": 5158, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17731.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "hely regular pa" }
+{ "l_orderkey": 5574, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18716.52, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old deposits int" }
+{ "l_orderkey": 5891, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21671.76, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1993-02-18", "l_receiptdate": "1993-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iresias cajole deposits. special, ir" }
+{ "l_orderkey": 32, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 43387.52, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "symptotes nag according to the ironic depo" }
+{ "l_orderkey": 35, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1995-12-25", "l_receiptdate": "1995-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly unti" }
+{ "l_orderkey": 64, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20707.68, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ch slyly final, thin platelets." }
+{ "l_orderkey": 195, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33526.72, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-02-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " cajole furiously bold i" }
+{ "l_orderkey": 195, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 40429.28, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-03-13", "l_receiptdate": "1994-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ggle fluffily foxes. fluffily ironic ex" }
+{ "l_orderkey": 772, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34512.8, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-06-13", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng ideas. special packages haggle alon" }
+{ "l_orderkey": 1092, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1972.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ans. slyly eve" }
+{ "l_orderkey": 1153, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14791.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-07-17", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uctions boost fluffily according to" }
+{ "l_orderkey": 1507, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38457.12, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-04", "l_commitdate": "1993-12-16", "l_receiptdate": "1993-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ly even instructions." }
+{ "l_orderkey": 1539, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10846.88, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-27", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly express requests. furiously " }
+{ "l_orderkey": 1570, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6902.56, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-01", "l_receiptdate": "1998-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "requests boost quickly re" }
+{ "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" }
+{ "l_orderkey": 1988, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8874.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1996-01-02", "l_receiptdate": "1996-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lar platelets. slyly ironic packa" }
+{ "l_orderkey": 2240, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9860.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "are across the ironic packages." }
+{ "l_orderkey": 2245, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 32540.64, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-06-11", "l_receiptdate": "1993-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ing to the carefully ruthless accounts" }
+{ "l_orderkey": 2371, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38457.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "tructions. regular, stealthy packages wak" }
+{ "l_orderkey": 2402, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42401.44, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-11-20", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly slyly blithe sheaves" }
+{ "l_orderkey": 2531, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19721.6, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "into beans. furious" }
+{ "l_orderkey": 2595, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ". final orbits cajole " }
+{ "l_orderkey": 2690, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "d accounts above the express req" }
+{ "l_orderkey": 2695, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39443.2, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-02", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-11-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ructions. pending" }
+{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44373.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-11-27", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "furiously special idea" }
+{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11832.96, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-22", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the slyly ironic dolphins; fin" }
+{ "l_orderkey": 3106, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21693.76, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "structions atop the blithely" }
+{ "l_orderkey": 3111, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4930.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-10-16", "l_receiptdate": "1995-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully even ideas" }
+{ "l_orderkey": 3397, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32540.64, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular accounts. blithely re" }
+{ "l_orderkey": 3490, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49304.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-06-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " haggle carefu" }
+{ "l_orderkey": 4166, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-01", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "unts. furiously express accounts w" }
+{ "l_orderkey": 4354, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake slyly eve" }
+{ "l_orderkey": 4835, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26624.16, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-13", "l_receiptdate": "1995-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts after the car" }
+{ "l_orderkey": 4896, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 20707.68, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-11-18", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly express deposits. carefully pending depo" }
+{ "l_orderkey": 4999, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-21", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole among the blithel" }
+{ "l_orderkey": 5088, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-05-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously final deposits. furiously re" }
+{ "l_orderkey": 5702, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36484.96, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1993-10-21", "l_receiptdate": "1994-01-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ix slyly. regular instructions slee" }
+{ "l_orderkey": 5861, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5916.48, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-05-18", "l_receiptdate": "1997-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "olites. slyly" }
+{ "l_orderkey": 5985, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3944.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-04", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ole along the quickly slow d" }
+{ "l_orderkey": 160, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21715.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-03-05", "l_receiptdate": "1997-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ncies about the request" }
+{ "l_orderkey": 451, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 987.08, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully ironic packages solve furiously " }
+{ "l_orderkey": 576, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1974.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-05-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ccounts along the ac" }
+{ "l_orderkey": 709, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6909.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-14", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " special orbits cajole " }
+{ "l_orderkey": 865, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14806.2, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-07-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " deposits sleep quickl" }
+{ "l_orderkey": 1093, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6909.56, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "bold deposits. blithely ironic depos" }
+{ "l_orderkey": 1156, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14806.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-21", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "the furiously pen" }
+{ "l_orderkey": 1252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12832.04, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sts dazzle" }
+{ "l_orderkey": 1540, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "carefully final packages; b" }
+{ "l_orderkey": 1766, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31586.56, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1996-11-11", "l_receiptdate": "1997-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ess accounts. stealthily ironic accou" }
+{ "l_orderkey": 2433, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38496.12, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final asy" }
+{ "l_orderkey": 2882, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46392.76, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-13", "l_commitdate": "1995-09-21", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l, special" }
+{ "l_orderkey": 3073, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9870.8, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " furiously caref" }
+{ "l_orderkey": 3107, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously final " }
+{ "l_orderkey": 3522, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ve the quickly special packages" }
+{ "l_orderkey": 3556, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27638.24, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully final instructions? ironic packa" }
+{ "l_orderkey": 3558, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7896.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-31", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "? even requests sle" }
+{ "l_orderkey": 4132, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17767.44, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-01", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y final de" }
+{ "l_orderkey": 4167, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16780.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly around the even instr" }
+{ "l_orderkey": 4196, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 42444.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es. slyly even " }
+{ "l_orderkey": 4294, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully; furiously ex" }
+{ "l_orderkey": 4545, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8883.72, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-02-23", "l_receiptdate": "1993-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "xpress accounts" }
+{ "l_orderkey": 4583, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 31586.56, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "across the pinto beans-- quickly" }
+{ "l_orderkey": 4610, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20728.68, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly special theodolites. even," }
+{ "l_orderkey": 4644, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9870.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the slow, final fo" }
+{ "l_orderkey": 4772, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 987.08, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ans. slyly even acc" }
+{ "l_orderkey": 4932, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4935.4, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-09-13", "l_receiptdate": "1993-10-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " haggle furiously. slyly ironic packages sl" }
+{ "l_orderkey": 5027, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 24677.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-11-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic ideas. requests sleep fluffily am" }
+{ "l_orderkey": 5056, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13819.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-09", "l_commitdate": "1997-04-13", "l_receiptdate": "1997-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts haggle carefully along the slyl" }
+{ "l_orderkey": 5252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-17", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "bold requests. furious" }
+{ "l_orderkey": 5281, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-31", "l_commitdate": "1995-12-23", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss the furiously " }
+{ "l_orderkey": 5572, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18754.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "es. final, final requests wake blithely ag" }
+{ "l_orderkey": 5605, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49354.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "instructions sleep carefully ironic req" }
+{ "l_orderkey": 5925, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-01-13", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the furiously" }
+{ "l_orderkey": 67, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 43475.52, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-04-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "se quickly above the even, express reques" }
+{ "l_orderkey": 99, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9880.8, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. requ" }
+{ "l_orderkey": 483, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " carefully express ins" }
+{ "l_orderkey": 612, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 988.08, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " requests." }
+{ "l_orderkey": 837, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23713.92, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p carefully. theodolites use. bold courts a" }
+{ "l_orderkey": 1031, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6916.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-11-11", "l_receiptdate": "1994-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "r instructions. car" }
+{ "l_orderkey": 1059, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44463.6, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "riously even theodolites. slyly regula" }
+{ "l_orderkey": 1792, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "final packages s" }
+{ "l_orderkey": 2181, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45451.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "osits. final packages sleep" }
+{ "l_orderkey": 2375, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41499.36, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "apades. idea" }
+{ "l_orderkey": 2469, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 34582.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-02-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ld packages haggle regular frets. fluffily " }
+{ "l_orderkey": 2595, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-03-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ctions. regula" }
+{ "l_orderkey": 2854, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28654.32, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y slyly ironic accounts. foxes haggle slyl" }
+{ "l_orderkey": 3045, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40511.28, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-11-24", "l_receiptdate": "1995-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final foxes. carefully ironic pinto b" }
+{ "l_orderkey": 3072, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 988.08, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " slyly ironic attainments. car" }
+{ "l_orderkey": 3394, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25690.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "its use furiously. even, even account" }
+{ "l_orderkey": 3588, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5928.48, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. fluffily fluf" }
+{ "l_orderkey": 3623, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-04", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic somas sleep fluffily" }
+{ "l_orderkey": 3842, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14821.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ave packages are slyl" }
+{ "l_orderkey": 3878, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12845.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-06-03", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep ruthlessly about the carefu" }
+{ "l_orderkey": 3970, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1976.16, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "carefully pending foxes wake blithely " }
+{ "l_orderkey": 4293, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24702.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "inal asympt" }
+{ "l_orderkey": 4640, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4940.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " warthogs against the regular" }
+{ "l_orderkey": 4640, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " accounts. unu" }
+{ "l_orderkey": 5158, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely fina" }
+{ "l_orderkey": 5184, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-27", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es above the care" }
+{ "l_orderkey": 5442, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 44463.6, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-02-24", "l_receiptdate": "1998-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "old slyly after " }
+{ "l_orderkey": 5703, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1976.16, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-07-26", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nts against the blithely sile" }
+{ "l_orderkey": 5957, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39523.2, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ironic asymptotes sleep blithely again" }
+{ "l_orderkey": 4, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "- quickly regular packages sleep. idly" }
+{ "l_orderkey": 34, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12858.04, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-23", "l_commitdate": "1998-09-14", "l_receiptdate": "1998-11-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nic accounts. deposits are alon" }
+{ "l_orderkey": 102, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36595.96, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-24", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ully across the ideas. final deposit" }
+{ "l_orderkey": 256, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21759.76, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1993-12-28", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke quickly ironic, ironic deposits. reg" }
+{ "l_orderkey": 1058, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4945.4, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-05-29", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "refully even requests boost along" }
+{ "l_orderkey": 1702, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33628.72, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-04", "l_commitdate": "1995-06-08", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y careful packages; dogged acco" }
+{ "l_orderkey": 2081, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "among the slyly express accounts. silen" }
+{ "l_orderkey": 2405, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17803.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "carefully ironic accounts. slyly " }
+{ "l_orderkey": 2688, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-18", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ithely final " }
+{ "l_orderkey": 2753, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-29", "l_receiptdate": "1994-02-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ans wake fluffily blithely iro" }
+{ "l_orderkey": 3110, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-01-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c theodolites a" }
+{ "l_orderkey": 3140, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9890.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "accounts. expres" }
+{ "l_orderkey": 3170, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26705.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "efully bold foxes. regular, ev" }
+{ "l_orderkey": 3429, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27694.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nstructions boost. thin" }
+{ "l_orderkey": 3622, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3956.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-02-19", "l_receiptdate": "1996-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lithely brave foxes. furi" }
+{ "l_orderkey": 3649, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22748.84, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-10-01", "l_receiptdate": "1994-09-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rs promise blithe" }
+{ "l_orderkey": 3940, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7912.64, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ions cajole furiously regular pinto beans. " }
+{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 37585.04, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly final requests. regu" }
+{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10879.88, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-31", "l_commitdate": "1992-10-04", "l_receiptdate": "1992-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits promise carefully even, regular e" }
+{ "l_orderkey": 4416, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2967.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests sleep along the " }
+{ "l_orderkey": 4549, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " requests wake. furiously even " }
+{ "l_orderkey": 4705, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-04-28", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "blithely. sly" }
+{ "l_orderkey": 5121, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45497.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "use express foxes. slyly " }
+{ "l_orderkey": 5186, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "capades. accounts sublate. pinto" }
+{ "l_orderkey": 5190, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44508.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y carefully final ideas. f" }
+{ "l_orderkey": 5568, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34617.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lyly. blit" }
+{ "l_orderkey": 5762, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al instructions. furiousl" }
+{ "l_orderkey": 5925, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49454.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-14", "l_commitdate": "1996-01-10", "l_receiptdate": "1996-02-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. stealthily express pains print bli" }
+{ "l_orderkey": 34, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21781.98, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-10-16", "l_receiptdate": "1998-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "thely slyly p" }
+{ "l_orderkey": 133, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10890.99, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-01-15", "l_receiptdate": "1998-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e quickly across the dolphins" }
+{ "l_orderkey": 359, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37623.42, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g furiously. regular, sile" }
+{ "l_orderkey": 646, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 16831.53, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1994-12-26", "l_receiptdate": "1995-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inal packages haggle carefully " }
+{ "l_orderkey": 710, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24752.25, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eas detect do" }
+{ "l_orderkey": 1058, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously f" }
+{ "l_orderkey": 1509, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 36633.33, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he slyly even deposits wake a" }
+{ "l_orderkey": 1568, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35643.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "platelets-- furiously sly excu" }
+{ "l_orderkey": 1601, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13861.26, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-11-22", "l_receiptdate": "1994-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he special, fin" }
+{ "l_orderkey": 1635, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-04-21", "l_receiptdate": "1997-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ravely carefully express " }
+{ "l_orderkey": 1827, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46534.23, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". pending courts about the even e" }
+{ "l_orderkey": 2272, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons along the blithely e" }
+{ "l_orderkey": 2311, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41583.78, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-06-27", "l_receiptdate": "1995-06-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gle furiously. bold " }
+{ "l_orderkey": 2400, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 990.09, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-18", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "silent deposits serve furious" }
+{ "l_orderkey": 2563, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49504.5, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly regular, regular excuses. bold plate" }
+{ "l_orderkey": 2592, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully special theodolites integrate " }
+{ "l_orderkey": 2593, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27722.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y even escapades shall" }
+{ "l_orderkey": 2628, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 49504.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-01-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "posits serve carefully toward " }
+{ "l_orderkey": 2917, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3960.36, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. unusual instruct" }
+{ "l_orderkey": 2944, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-30", "l_commitdate": "1997-11-03", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fluffily blithely express pea" }
+{ "l_orderkey": 2978, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas promise slyly" }
+{ "l_orderkey": 3105, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-28", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. blithely unusual ideas was after" }
+{ "l_orderkey": 3143, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21781.98, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-03-26", "l_receiptdate": "1993-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, special instructions nag " }
+{ "l_orderkey": 3170, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 31682.88, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggle about the furiously r" }
+{ "l_orderkey": 3335, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46534.23, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-12-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly special ideas." }
+{ "l_orderkey": 3364, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48514.41, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-17", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "d accounts? caref" }
+{ "l_orderkey": 3461, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15841.44, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-04-29", "l_receiptdate": "1993-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " pending deposi" }
+{ "l_orderkey": 3552, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns after the blithely reg" }
+{ "l_orderkey": 3559, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "l, regular accounts wake flu" }
+{ "l_orderkey": 3713, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-08-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "totes. carefully special theodolites s" }
+{ "l_orderkey": 3719, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12871.17, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously. regular dep" }
+{ "l_orderkey": 4067, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39603.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar theodolites nag blithely above the" }
+{ "l_orderkey": 4162, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-25", "l_commitdate": "1992-04-25", "l_receiptdate": "1992-03-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nding pinto beans haggle blithe" }
+{ "l_orderkey": 4386, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-04-09", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " deposits use according to the pending, " }
+{ "l_orderkey": 4576, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4950.45, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express, special asymptote" }
+{ "l_orderkey": 5056, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22772.07, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ickly regular requests cajole. depos" }
+{ "l_orderkey": 5186, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y regular notornis k" }
+{ "l_orderkey": 5445, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32672.97, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-10-14", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ests. final instructions" }
+{ "l_orderkey": 5474, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-06-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nstructions. furio" }
+{ "l_orderkey": 5536, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13861.26, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "instructions sleep " }
+{ "l_orderkey": 5636, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20791.89, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-05-11", "l_receiptdate": "1995-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " are furiously unusual " }
+{ "l_orderkey": 5669, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-05", "l_commitdate": "1996-06-10", "l_receiptdate": "1996-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "to beans against the regular depo" }
+{ "l_orderkey": 5670, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26732.43, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " ideas promise bli" }
+{ "l_orderkey": 5829, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41583.78, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pearls. slyly bold deposits solve final" }
+{ "l_orderkey": 5922, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34653.15, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-21", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. regu" }
+{ "l_orderkey": 5926, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-07-20", "l_receiptdate": "1994-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle furiously express foxes. bo" }
+{ "l_orderkey": 5927, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rding to the special, final decoy" }
+{ "l_orderkey": 5986, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-09-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "structions! furiously pending instructi" }
+{ "l_orderkey": 613, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16848.53, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-10-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar dependencie" }
+{ "l_orderkey": 741, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21803.98, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ven deposits about the regular, ironi" }
+{ "l_orderkey": 806, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3964.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-09", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eans. quickly ironic ideas " }
+{ "l_orderkey": 897, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14866.35, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-25", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r ideas. slyly spec" }
+{ "l_orderkey": 961, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17839.62, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-08-14", "l_receiptdate": "1995-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "rmanent foxes haggle speci" }
+{ "l_orderkey": 1794, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 36670.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-21", "l_receiptdate": "1998-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. pinto" }
+{ "l_orderkey": 2117, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23786.16, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-27", "l_receiptdate": "1997-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "thely slyly pending platelets. ironic, " }
+{ "l_orderkey": 2181, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14866.35, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-10-27", "l_receiptdate": "1995-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e above the fluffily regul" }
+{ "l_orderkey": 2275, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1992-12-10", "l_receiptdate": "1993-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ost across the never express instruction" }
+{ "l_orderkey": 2373, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4955.45, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-06-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily blithely ironic requests" }
+{ "l_orderkey": 2407, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9910.9, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " pending instructions. theodolites x-" }
+{ "l_orderkey": 2503, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26759.43, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-07-24", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lly even p" }
+{ "l_orderkey": 2691, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "leep alongside of the accounts. slyly ironi" }
+{ "l_orderkey": 2818, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6937.63, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-03-09", "l_receiptdate": "1995-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly according to the r" }
+{ "l_orderkey": 3015, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are slyly carefully special pinto bea" }
+{ "l_orderkey": 3043, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4955.45, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-07-02", "l_receiptdate": "1992-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ake blithely re" }
+{ "l_orderkey": 3077, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39643.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-22", "l_commitdate": "1997-09-19", "l_receiptdate": "1997-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "to the enticing packag" }
+{ "l_orderkey": 3105, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " detect slyly. blithely unusual requests ar" }
+{ "l_orderkey": 3360, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28741.61, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-02-25", "l_receiptdate": "1998-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "press asymptotes. furiously final " }
+{ "l_orderkey": 3558, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21803.98, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-04-19", "l_receiptdate": "1996-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "refully ironic theodolites are fu" }
+{ "l_orderkey": 3588, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-03", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "special pinto beans cajole slyly. slyly " }
+{ "l_orderkey": 4321, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3964.36, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-10-06", "l_receiptdate": "1994-09-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic deposi" }
+{ "l_orderkey": 4448, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 40634.69, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pon the permanently even excuses nag " }
+{ "l_orderkey": 4486, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "to the furious, regular foxes play abov" }
+{ "l_orderkey": 4835, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2973.27, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "etimes final pac" }
+{ "l_orderkey": 5158, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 37661.42, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uffily regular ac" }
+{ "l_orderkey": 5186, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30723.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-09-26", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " accounts use furiously slyly spe" }
+{ "l_orderkey": 5376, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43607.96, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithe packages detect final theodolites. f" }
+{ "l_orderkey": 164, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25794.34, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s. blithely special courts are blithel" }
+{ "l_orderkey": 295, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25794.34, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts above the slyly regular requests x-ray q" }
+{ "l_orderkey": 487, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46628.23, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-10-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tions. blithely reg" }
+{ "l_orderkey": 992, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6944.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-12-24", "l_receiptdate": "1997-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ideas haggle. special theodolit" }
+{ "l_orderkey": 1122, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7936.72, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c foxes are along the slyly r" }
+{ "l_orderkey": 1153, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-09", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "special instructions are. unusual, final du" }
+{ "l_orderkey": 1186, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s haggle furiously; slyl" }
+{ "l_orderkey": 1252, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s are. slyly final requests among the" }
+{ "l_orderkey": 1280, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 18849.71, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-02-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lyly along the furiously regular " }
+{ "l_orderkey": 1474, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-23", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-02-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "after the special" }
+{ "l_orderkey": 1986, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-06-28", "l_receiptdate": "1994-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sleep furiously fluffily final" }
+{ "l_orderkey": 2144, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32738.97, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-04-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic excuses haggle final dependencies. " }
+{ "l_orderkey": 2658, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " dependencies. blithely pending foxes abou" }
+{ "l_orderkey": 2724, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46628.23, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-13", "l_receiptdate": "1994-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual patterns nag. special p" }
+{ "l_orderkey": 2755, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18849.71, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-11", "l_commitdate": "1992-03-15", "l_receiptdate": "1992-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "furiously special deposits" }
+{ "l_orderkey": 2948, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 48612.41, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ress requests. furiously blithe foxes " }
+{ "l_orderkey": 3010, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 37699.42, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-03-16", "l_receiptdate": "1996-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "accounts ar" }
+{ "l_orderkey": 3174, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20833.89, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-20", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "iously. idly bold theodolites a" }
+{ "l_orderkey": 3490, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-08-06", "l_receiptdate": "1997-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". even requests cajol" }
+{ "l_orderkey": 3776, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48612.41, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-01-06", "l_receiptdate": "1993-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es: careful warthogs haggle fluffi" }
+{ "l_orderkey": 3908, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49604.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " even accounts wake " }
+{ "l_orderkey": 3911, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uctions. blithely regula" }
+{ "l_orderkey": 4580, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nticingly final packag" }
+{ "l_orderkey": 4615, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9920.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-20", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sits. slyly express deposits are" }
+{ "l_orderkey": 4801, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38691.51, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "warhorses wake never for the care" }
+{ "l_orderkey": 5094, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-24", "l_receiptdate": "1993-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s cajole quickly against the furiously ex" }
+{ "l_orderkey": 5571, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-11", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "uests haggle furiously pending d" }
+{ "l_orderkey": 5606, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33731.06, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uses. slyly final " }
+{ "l_orderkey": 5824, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31746.88, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-16", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ven requests. " }
+{ "l_orderkey": 5987, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-30", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "theodolites wake above the furiously b" }
+{ "l_orderkey": 69, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-07-26", "l_receiptdate": "1994-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "tect regular, speci" }
+{ "l_orderkey": 193, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8937.81, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-08", "l_receiptdate": "1993-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "against the fluffily regular d" }
+{ "l_orderkey": 384, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10923.99, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nic excuses are furiously above the blith" }
+{ "l_orderkey": 548, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-24", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "courts boost care" }
+{ "l_orderkey": 581, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13903.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ". deposits s" }
+{ "l_orderkey": 640, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48661.41, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-04-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s haggle slyly" }
+{ "l_orderkey": 1124, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34758.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-25", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ut the slyly bold pinto beans; fi" }
+{ "l_orderkey": 1283, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46675.23, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even instructions boost slyly blithely " }
+{ "l_orderkey": 1508, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42702.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-01", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ndencies h" }
+{ "l_orderkey": 1572, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9930.9, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-05-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " accounts affix slyly. " }
+{ "l_orderkey": 1637, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely a" }
+{ "l_orderkey": 1698, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5958.54, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-21", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending packages affix ne" }
+{ "l_orderkey": 1702, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27806.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-07-26", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts haggle along the packa" }
+{ "l_orderkey": 2530, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-03-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ng platelets wake s" }
+{ "l_orderkey": 2881, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20854.89, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-07-03", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "hely express Tiresias. final dependencies " }
+{ "l_orderkey": 2917, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35751.24, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly ironic d" }
+{ "l_orderkey": 3138, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6951.63, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-03-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely quickly even packages. packages" }
+{ "l_orderkey": 3269, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the special packages. " }
+{ "l_orderkey": 3490, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inal deposits use furiousl" }
+{ "l_orderkey": 3493, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30785.79, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ructions. slyly regular accounts across the" }
+{ "l_orderkey": 3619, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17875.62, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites " }
+{ "l_orderkey": 3621, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12910.17, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-06-30", "l_receiptdate": "1993-09-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r the unusual packages. brave theodoli" }
+{ "l_orderkey": 3654, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28799.61, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "odolites detect. quickly r" }
+{ "l_orderkey": 4066, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "quests. slyly regu" }
+{ "l_orderkey": 4166, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es along the furiously regular acc" }
+{ "l_orderkey": 4487, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24827.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g the final instructions. slyly c" }
+{ "l_orderkey": 4647, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15889.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "o beans about the fluffily special the" }
+{ "l_orderkey": 4928, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19861.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "quiet theodolites ca" }
+{ "l_orderkey": 5509, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29792.7, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-23", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "counts haggle pinto beans. furiously " }
+{ "l_orderkey": 5857, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 48661.41, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-12-12", "l_receiptdate": "1998-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "egular pinto beans" }
+{ "l_orderkey": 5859, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39723.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-08-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dependenci" }
+{ "l_orderkey": 193, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22864.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even accounts wake blithely bold" }
+{ "l_orderkey": 195, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-03-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rts detect in place of t" }
+{ "l_orderkey": 224, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 44734.05, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "leep furiously regular requests. furiousl" }
+{ "l_orderkey": 416, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24852.25, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-11-26", "l_receiptdate": "1993-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y final theodolites about" }
+{ "l_orderkey": 1218, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "dolphins. theodolites beyond th" }
+{ "l_orderkey": 1281, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1988.18, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly unusual requests. final reques" }
+{ "l_orderkey": 1696, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 42745.87, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-03-29", "l_receiptdate": "1998-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "arefully regular dep" }
+{ "l_orderkey": 1697, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48710.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1997-01-02", "l_receiptdate": "1996-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar foxes. fluffily furious ideas doubt qu" }
+{ "l_orderkey": 1762, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34793.15, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ind quickly. accounts ca" }
+{ "l_orderkey": 2084, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8946.81, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-03-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heaves boost slyly after the pla" }
+{ "l_orderkey": 2182, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33799.06, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-28", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " slow tithes. ironi" }
+{ "l_orderkey": 2437, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e of the bold, dogged requests" }
+{ "l_orderkey": 2468, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-16", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "unusual theodolites su" }
+{ "l_orderkey": 2533, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 13917.26, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-06", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ut the pending, special depos" }
+{ "l_orderkey": 2915, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11929.08, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-18", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-07-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "accounts. slyly final" }
+{ "l_orderkey": 2946, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47716.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-03-31", "l_receiptdate": "1996-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oss the platelets. furi" }
+{ "l_orderkey": 3013, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30816.79, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y furious depen" }
+{ "l_orderkey": 3173, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1988.18, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular pearls" }
+{ "l_orderkey": 3585, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6958.63, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dependencies sleep un" }
+{ "l_orderkey": 3650, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 26840.43, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-07-23", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular requests snooze fluffily regular pi" }
+{ "l_orderkey": 3654, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33799.06, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the quick" }
+{ "l_orderkey": 3778, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e the furiously ironi" }
+{ "l_orderkey": 4001, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely ironic d" }
+{ "l_orderkey": 4039, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37775.42, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1997-12-31", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "sual asymptotes. ironic deposits nag aft" }
+{ "l_orderkey": 4353, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21869.98, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ent packages. accounts are slyly. " }
+{ "l_orderkey": 4416, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36781.33, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fluffily ironic " }
+{ "l_orderkey": 4642, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-06-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ily pending accounts hag" }
+{ "l_orderkey": 5189, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48710.41, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests " }
+{ "l_orderkey": 5474, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9940.9, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "pinto bean" }
+{ "l_orderkey": 5571, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30816.79, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1993-01-18", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uffily even accounts. quickly re" }
+{ "l_orderkey": 5574, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13917.26, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-20", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " use slyly carefully special requests? slyl" }
+{ "l_orderkey": 5954, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19881.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-02-05", "l_receiptdate": "1992-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " accounts wake carefu" }
+{ "l_orderkey": 7, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 45774.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-27", "l_receiptdate": "1996-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual reques" }
+{ "l_orderkey": 39, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 39803.6, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-22", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "quickly ironic fox" }
+{ "l_orderkey": 68, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19901.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " excuses integrate fluffily " }
+{ "l_orderkey": 453, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 27862.52, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final dependencies. slyly special pl" }
+{ "l_orderkey": 610, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4975.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-11", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "n pinto beans. iro" }
+{ "l_orderkey": 641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39803.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-10-20", "l_receiptdate": "1993-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lets. furiously regular requests cajo" }
+{ "l_orderkey": 801, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20896.89, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "wake silently furiously idle deposits. " }
+{ "l_orderkey": 838, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-26", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ets haggle furiously furiously regular r" }
+{ "l_orderkey": 1124, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-07", "l_commitdate": "1998-08-31", "l_receiptdate": "1998-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly bold accou" }
+{ "l_orderkey": 1287, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9950.9, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-08", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely alongside of the unusual, ironic pa" }
+{ "l_orderkey": 1348, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12936.17, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely r" }
+{ "l_orderkey": 1667, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 47764.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-27", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "tes sleep furiously. carefully eve" }
+{ "l_orderkey": 1794, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2985.27, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " sentiments according to the q" }
+{ "l_orderkey": 1831, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ests. express pinto beans abou" }
+{ "l_orderkey": 2176, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13931.26, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-07", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely ironic platelets " }
+{ "l_orderkey": 2273, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7960.72, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-01-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "dependencies. slyly ir" }
+{ "l_orderkey": 2434, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-02", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " furiously express packages. ironic, pend" }
+{ "l_orderkey": 2628, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1994-01-08", "l_receiptdate": "1993-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usual packages sleep about the fina" }
+{ "l_orderkey": 3235, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42788.87, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly final instru" }
+{ "l_orderkey": 3271, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13931.26, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-24", "l_commitdate": "1992-02-14", "l_receiptdate": "1992-03-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending, even packa" }
+{ "l_orderkey": 3328, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41793.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-12-20", "l_receiptdate": "1992-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic requests" }
+{ "l_orderkey": 3430, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4975.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "even accounts haggle slyly bol" }
+{ "l_orderkey": 3460, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49754.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e slyly about the sly" }
+{ "l_orderkey": 3461, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 40798.69, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-19", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-02-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heodolites. blithely ironi" }
+{ "l_orderkey": 3590, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18906.71, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "special pinto beans. blithely reg" }
+{ "l_orderkey": 4327, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17911.62, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-20", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y final excuses. ironic, special requests a" }
+{ "l_orderkey": 4484, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3980.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "packages de" }
+{ "l_orderkey": 4641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 38808.51, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-10", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the bold reque" }
+{ "l_orderkey": 4772, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30847.79, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ests are thinly. furiously unusua" }
+{ "l_orderkey": 5317, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18906.71, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-12-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "onic requests boost bli" }
+{ "l_orderkey": 5799, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40798.69, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-10-31", "l_receiptdate": "1995-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al accounts sleep ruthlessl" }
+{ "l_orderkey": 260, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 43827.96, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "above the blithely ironic instr" }
+{ "l_orderkey": 323, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17929.62, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "posits cajole furiously pinto beans. " }
+{ "l_orderkey": 453, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 44824.05, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-06-29", "l_receiptdate": "1997-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ironic foxes. slyly pending depos" }
+{ "l_orderkey": 645, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. slyly iron" }
+{ "l_orderkey": 742, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14941.35, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-26", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-03-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "blithely unusual pinto" }
+{ "l_orderkey": 1063, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41835.78, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-10", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tructions about the blithely ex" }
+{ "l_orderkey": 1159, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6972.63, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-12-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "olve somet" }
+{ "l_orderkey": 1315, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26894.43, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-07-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "latelets. fluffily ironic account" }
+{ "l_orderkey": 1441, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49804.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " requests. blithely e" }
+{ "l_orderkey": 1671, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-09-19", "l_receiptdate": "1996-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lyly regular ac" }
+{ "l_orderkey": 1760, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37851.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tions. blithely regular orbits against the " }
+{ "l_orderkey": 1920, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23906.16, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-08-23", "l_receiptdate": "1998-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "thely. bold, pend" }
+{ "l_orderkey": 2052, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "final requests. stealt" }
+{ "l_orderkey": 2181, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ward the quietly even requests. ir" }
+{ "l_orderkey": 3172, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-26", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s are slyly thin package" }
+{ "l_orderkey": 3268, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 996.09, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". ironic, bold requests use carefull" }
+{ "l_orderkey": 3590, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42831.87, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s could have to use" }
+{ "l_orderkey": 3619, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-22", "l_commitdate": "1996-12-21", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " waters. furiously even deposits " }
+{ "l_orderkey": 3943, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-03", "l_receiptdate": "1996-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully ironic " }
+{ "l_orderkey": 3971, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e slyly final dependencies x-ray " }
+{ "l_orderkey": 4067, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13945.26, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ructions. quickly ironic accounts detect " }
+{ "l_orderkey": 4067, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11953.08, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1992-11-28", "l_receiptdate": "1992-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lly slyly even theodol" }
+{ "l_orderkey": 4134, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33867.06, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual asymptotes wake carefully alo" }
+{ "l_orderkey": 4197, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-09-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l instructions print slyly past the reg" }
+{ "l_orderkey": 4225, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-08-31", "l_receiptdate": "1997-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". quickly b" }
+{ "l_orderkey": 4229, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43827.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. carefully e" }
+{ "l_orderkey": 4262, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4980.45, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-27", "l_commitdate": "1996-09-05", "l_receiptdate": "1996-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely final asymptotes integrate" }
+{ "l_orderkey": 4450, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44824.05, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-01", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "express ideas are furiously regular" }
+{ "l_orderkey": 4482, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31874.88, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eans wake according " }
+{ "l_orderkey": 4486, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-05-24", "l_receiptdate": "1998-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ts around the quiet packages ar" }
+{ "l_orderkey": 4678, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12949.17, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-03", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "its. carefully final fr" }
+{ "l_orderkey": 5026, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12949.17, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-23", "l_commitdate": "1997-11-02", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "endencies sleep carefully alongs" }
+{ "l_orderkey": 5185, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29882.7, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-17", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ress packages are furiously" }
+{ "l_orderkey": 5383, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11953.08, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-02", "l_commitdate": "1995-08-16", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y regular instructi" }
+{ "l_orderkey": 5411, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16933.53, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly slyly even deposits. carefully b" }
+{ "l_orderkey": 5477, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "telets wake blithely ab" }
+{ "l_orderkey": 5541, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38847.51, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-12-27", "l_receiptdate": "1997-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ding theodolites haggle against the slyly " }
+{ "l_orderkey": 5637, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21913.98, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-09-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nding requests are ca" }
+{ "l_orderkey": 71, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32903.97, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-04-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " serve quickly fluffily bold deposi" }
+{ "l_orderkey": 226, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3988.36, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-04-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "c foxes integrate carefully against th" }
+{ "l_orderkey": 261, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 19941.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ing to the special, ironic deposi" }
+{ "l_orderkey": 355, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40880.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " deposits. carefully r" }
+{ "l_orderkey": 358, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14956.35, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-12-17", "l_receiptdate": "1993-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "out the blithely ironic deposits slee" }
+{ "l_orderkey": 387, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39883.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-04-18", "l_receiptdate": "1997-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " quickly ironic platelets are slyly. fluff" }
+{ "l_orderkey": 484, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9970.9, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "x fluffily carefully regular" }
+{ "l_orderkey": 871, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-25", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-03-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "coys dazzle slyly slow notornis. f" }
+{ "l_orderkey": 961, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41877.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ests do cajole blithely. furiously bo" }
+{ "l_orderkey": 1057, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-04-30", "l_receiptdate": "1992-06-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y slyly express theodolites. slyly bo" }
+{ "l_orderkey": 1154, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4985.45, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furiously " }
+{ "l_orderkey": 1216, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7976.72, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-01", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " of the carefully express" }
+{ "l_orderkey": 1606, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19941.8, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fily carefu" }
+{ "l_orderkey": 1698, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43871.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-05-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts wake slyly after t" }
+{ "l_orderkey": 1856, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46863.23, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ingly blithe theodolites. slyly pending " }
+{ "l_orderkey": 2048, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes. idly ironic packages nag" }
+{ "l_orderkey": 2208, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-07-19", "l_receiptdate": "1995-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ding waters lose. furiously regu" }
+{ "l_orderkey": 2241, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-07-30", "l_receiptdate": "1993-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss accounts engage furiously. slyly even re" }
+{ "l_orderkey": 2278, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21935.98, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-07-14", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ep regular accounts. blithely even" }
+{ "l_orderkey": 2850, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42874.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1996-11-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "unusual accounts" }
+{ "l_orderkey": 2945, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 44869.05, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1996-03-25", "l_receiptdate": "1996-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ainst the final packages" }
+{ "l_orderkey": 3072, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uests. ironic, ironic depos" }
+{ "l_orderkey": 3393, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24927.25, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ng excuses" }
+{ "l_orderkey": 3430, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40880.69, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cuses. silent excuses h" }
+{ "l_orderkey": 3655, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 997.09, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "arefully slow pinto beans are" }
+{ "l_orderkey": 3715, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12962.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e quickly ironic" }
+{ "l_orderkey": 4035, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3988.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-04-23", "l_receiptdate": "1992-04-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ilent, even pear" }
+{ "l_orderkey": 4065, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 32903.97, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ain blithely " }
+{ "l_orderkey": 4644, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15953.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-21", "l_receiptdate": "1998-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lar excuses across the " }
+{ "l_orderkey": 4800, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic dependenc" }
+{ "l_orderkey": 4934, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " ideas cajol" }
+{ "l_orderkey": 5029, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1994.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages. furiously ironi" }
+{ "l_orderkey": 5095, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14956.35, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " to the packages wake sly" }
+{ "l_orderkey": 5121, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26921.43, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly silent theodolit" }
+{ "l_orderkey": 5280, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15953.44, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-04-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " foxes are furiously. theodoli" }
+{ "l_orderkey": 5286, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-23", "l_commitdate": "1997-12-10", "l_receiptdate": "1997-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y express instructions sleep carefull" }
+{ "l_orderkey": 5412, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25924.34, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-22", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-02-17", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " the blithel" }
+{ "l_orderkey": 5537, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37889.42, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s above the carefully ironic deposits " }
+{ "l_orderkey": 5987, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 36892.33, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "le furiously carefully special " }
+{ "l_orderkey": 192, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-05", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly pending theodolites haggle quickly fluf" }
+{ "l_orderkey": 678, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15969.44, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-04-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " about the " }
+{ "l_orderkey": 736, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13973.26, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-10-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nstructions." }
+{ "l_orderkey": 864, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6986.63, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-07", "l_receiptdate": "1997-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven requests should sleep along " }
+{ "l_orderkey": 963, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47908.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ages. quickly express deposits cajole pe" }
+{ "l_orderkey": 1125, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 28944.61, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-29", "l_commitdate": "1994-12-20", "l_receiptdate": "1994-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " platelets wake against the carefully i" }
+{ "l_orderkey": 1159, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-09", "l_commitdate": "1992-12-07", "l_receiptdate": "1992-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "h furiousl" }
+{ "l_orderkey": 1348, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1996.18, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final packages use fluffily express ac" }
+{ "l_orderkey": 1510, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e of the unusual accounts. stealthy deposit" }
+{ "l_orderkey": 1511, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28944.61, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s cajole furiously against " }
+{ "l_orderkey": 1888, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26948.43, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-02-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". carefully special dolphins sle" }
+{ "l_orderkey": 2561, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4990.45, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p ironic, regular pinto beans." }
+{ "l_orderkey": 2883, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-03", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even requests cajole. special, regular " }
+{ "l_orderkey": 3463, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42917.87, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " across the " }
+{ "l_orderkey": 3526, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-05-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. furiously regular d" }
+{ "l_orderkey": 3617, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15969.44, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " slyly on th" }
+{ "l_orderkey": 3617, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31938.88, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-05-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uriously against the express accounts. ex" }
+{ "l_orderkey": 3777, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13973.26, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-05-31", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ording to the iro" }
+{ "l_orderkey": 3778, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-02", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits. theodol" }
+{ "l_orderkey": 4225, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts are requests. even, bold depos" }
+{ "l_orderkey": 4390, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31938.88, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-15", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions across" }
+{ "l_orderkey": 4417, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34933.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "slyly regular, silent courts. even packag" }
+{ "l_orderkey": 4421, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36929.33, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l accounts. ironic request" }
+{ "l_orderkey": 4932, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "as. special depende" }
+{ "l_orderkey": 4960, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7984.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-14", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "as. busily regular packages nag. " }
+{ "l_orderkey": 5027, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5988.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-28", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-10-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar, ironic deposi" }
+{ "l_orderkey": 5414, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts sleep sl" }
+{ "l_orderkey": 5603, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49904.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-06", "l_commitdate": "1992-08-20", "l_receiptdate": "1992-10-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "final theodolites accor" }
+{ "l_orderkey": 5696, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19961.8, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "silent, pending ideas sleep fluffil" }
+{ "l_orderkey": 197, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38964.51, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "press accounts. daringly sp" }
+{ "l_orderkey": 259, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13987.26, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ons against the express acco" }
+{ "l_orderkey": 288, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35967.24, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-22", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "yly pending excu" }
+{ "l_orderkey": 646, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33969.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "slow accounts. fluffily idle instructions" }
+{ "l_orderkey": 803, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20980.89, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-25", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ironic packages cajole slyly. un" }
+{ "l_orderkey": 1251, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 36966.33, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-12-01", "l_receiptdate": "1998-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "finally bold requests" }
+{ "l_orderkey": 1409, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22979.07, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ions. slyly ironic packages wake quick" }
+{ "l_orderkey": 1732, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35967.24, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-04-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ve the accounts. slowly ironic multip" }
+{ "l_orderkey": 1893, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42960.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "he carefully regular " }
+{ "l_orderkey": 2149, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9990.9, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eposits sleep above" }
+{ "l_orderkey": 2599, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 28973.61, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly express dolphins. special, " }
+{ "l_orderkey": 2823, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17983.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "eas. decoys cajole deposi" }
+{ "l_orderkey": 2982, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12988.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "regular deposits unwind alongside " }
+{ "l_orderkey": 3303, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 36966.33, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-16", "l_commitdate": "1998-03-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " carefully ironic asympt" }
+{ "l_orderkey": 3462, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1998.18, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-10", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nic packages. even accounts alongside " }
+{ "l_orderkey": 3586, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " ironic pinto beans cajole carefully theo" }
+{ "l_orderkey": 4002, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3996.36, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ccording to the careful" }
+{ "l_orderkey": 4290, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2997.27, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-03-07", "l_receiptdate": "1995-04-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lar platelets cajole" }
+{ "l_orderkey": 4998, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-03", "l_receiptdate": "1992-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions nag quickly according to the theodolit" }
+{ "l_orderkey": 5447, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30971.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes sleep. blithely unusual accounts det" }
+{ "l_orderkey": 5509, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16984.53, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts wake ar" }
+{ "l_orderkey": 5921, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43959.96, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ain about the special" }
+{ "l_orderkey": 166, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41004.1, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "hily along the blithely pending fo" }
+{ "l_orderkey": 292, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " bold, pending theodolites u" }
+{ "l_orderkey": 641, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1000.1, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " nag across the regular foxes." }
+{ "l_orderkey": 675, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15001.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits after the furio" }
+{ "l_orderkey": 773, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5000.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar requests. regular, thin packages u" }
+{ "l_orderkey": 930, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21002.1, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "foxes. regular deposits integrate carefu" }
+{ "l_orderkey": 933, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26002.6, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the deposits affix slyly after t" }
+{ "l_orderkey": 1027, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13001.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ily ironic ideas use" }
+{ "l_orderkey": 1028, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8000.8, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-03-28", "l_receiptdate": "1994-02-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e carefully final packages. furiously fi" }
+{ "l_orderkey": 1152, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25002.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-20", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "efully ironic accounts. sly instructions wa" }
+{ "l_orderkey": 1223, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28002.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " quickly ironic requests. furious" }
+{ "l_orderkey": 1445, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-02-22", "l_receiptdate": "1995-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al accounts use furiously a" }
+{ "l_orderkey": 1606, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ously final requests. slowly ironic ex" }
+{ "l_orderkey": 1828, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33003.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s boost carefully. pending d" }
+{ "l_orderkey": 1857, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41004.1, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " the slyly" }
+{ "l_orderkey": 1890, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43004.3, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p ironic, express accounts. fu" }
+{ "l_orderkey": 2022, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36003.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly after the foxes. regular, final inst" }
+{ "l_orderkey": 2470, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-01", "l_receiptdate": "1997-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages " }
+{ "l_orderkey": 2567, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 32003.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-24", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even, iro" }
+{ "l_orderkey": 2785, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34003.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly final packages haggl" }
+{ "l_orderkey": 2852, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12001.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "le. request" }
+{ "l_orderkey": 3170, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21002.1, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-09", "l_commitdate": "1998-01-31", "l_receiptdate": "1997-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "o beans. carefully final requests dou" }
+{ "l_orderkey": 3233, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2000.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1995-01-02", "l_receiptdate": "1995-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " across the bold packages" }
+{ "l_orderkey": 3461, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49004.9, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-16", "l_receiptdate": "1993-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ual request" }
+{ "l_orderkey": 3649, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-20", "l_receiptdate": "1994-07-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "c accounts. quickly final theodo" }
+{ "l_orderkey": 3683, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-07-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "xpress accounts sleep slyly re" }
+{ "l_orderkey": 3777, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11001.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-06-05", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ld ideas. even theodolites" }
+{ "l_orderkey": 3808, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34003.4, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-07-22", "l_receiptdate": "1994-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " pearls will have to " }
+{ "l_orderkey": 4134, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ironic pin" }
+{ "l_orderkey": 4262, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 28002.8, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-09-06", "l_receiptdate": "1996-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ironic, regular depend" }
+{ "l_orderkey": 4738, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the blithely ironic braids sleep slyly" }
+{ "l_orderkey": 4739, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly even packages use across th" }
+{ "l_orderkey": 4742, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-12", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ke carefully. do" }
+{ "l_orderkey": 4839, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19001.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits sublate furiously ir" }
+{ "l_orderkey": 4928, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4000.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bout the slyly final accounts. carefull" }
+{ "l_orderkey": 4961, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10001.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests. regular, ironic ideas at the ironi" }
+{ "l_orderkey": 5509, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-24", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "counts sleep. f" }
+{ "l_orderkey": 5633, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48004.8, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even courts haggle slyly at the requ" }
+{ "l_orderkey": 5799, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " furiously s" }
+{ "l_orderkey": 5920, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42004.2, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lar, ironic dependencies sno" }
+{ "l_orderkey": 5954, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35003.5, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tions maintain slyly. furious" }
+{ "l_orderkey": 420, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5005.5, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-04", "l_commitdate": "1996-01-02", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole blit" }
+{ "l_orderkey": 581, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49053.9, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly regular pinto beans acr" }
+{ "l_orderkey": 644, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-07-26", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "iously ironic pinto beans. bold packa" }
+{ "l_orderkey": 742, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48052.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-04-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " platelets " }
+{ "l_orderkey": 1123, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38041.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " blithely carefully unusual reques" }
+{ "l_orderkey": 1186, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ely alongside of the blithel" }
+{ "l_orderkey": 1571, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 48052.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-28", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly pending p" }
+{ "l_orderkey": 1893, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18019.8, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g packages. fluffily final reques" }
+{ "l_orderkey": 1958, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31034.1, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "r deposits c" }
+{ "l_orderkey": 1990, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46050.6, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-03-14", "l_receiptdate": "1995-01-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar sentiments." }
+{ "l_orderkey": 2048, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12013.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " even theodoli" }
+{ "l_orderkey": 2053, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-25", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-05-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly ironic foxes haggle slyly speci" }
+{ "l_orderkey": 2147, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4004.4, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "mong the blithely special" }
+{ "l_orderkey": 2371, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "requests. regular pinto beans wake. car" }
+{ "l_orderkey": 2464, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-02", "l_receiptdate": "1998-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sts. slyly close ideas shall h" }
+{ "l_orderkey": 2599, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " express accoun" }
+{ "l_orderkey": 3044, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10011.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly ironic requests. s" }
+{ "l_orderkey": 3683, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35038.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " the furiously expr" }
+{ "l_orderkey": 3905, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uses are care" }
+{ "l_orderkey": 4355, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 47051.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-29", "l_receiptdate": "1997-01-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e. realms integrate " }
+{ "l_orderkey": 4390, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42046.2, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-16", "l_receiptdate": "1995-06-17", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "arefully even accoun" }
+{ "l_orderkey": 4673, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-10-31", "l_receiptdate": "1997-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " gifts cajole dari" }
+{ "l_orderkey": 4965, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27029.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "efully final foxes" }
+{ "l_orderkey": 5062, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9009.9, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " silent theodolites wake. c" }
+{ "l_orderkey": 5126, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1992-12-19", "l_receiptdate": "1993-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e silently. ironic, unusual accounts" }
+{ "l_orderkey": 5572, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14015.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he fluffily express packages. fluffily fina" }
+{ "l_orderkey": 5665, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32035.2, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-09-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "f the slyly even requests! regular request" }
+{ "l_orderkey": 5764, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28030.8, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sleep furi" }
+{ "l_orderkey": 167, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-19", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sly during the u" }
+{ "l_orderkey": 198, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 33069.3, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ending foxes acr" }
+{ "l_orderkey": 674, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-25", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-11-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ve the quickly even deposits. blithe" }
+{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46096.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly bold deposits cajole according to" }
+{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24050.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix slyly. furiously i" }
+{ "l_orderkey": 897, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "into beans. slyly special fox" }
+{ "l_orderkey": 929, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7014.7, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-12-19", "l_receiptdate": "1993-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ithely. slyly c" }
+{ "l_orderkey": 1538, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uses maintain blithely. fluffily" }
+{ "l_orderkey": 1958, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4008.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-12-09", "l_receiptdate": "1995-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he slyly even dependencies " }
+{ "l_orderkey": 2119, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36075.6, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly bold foxes. ironic accoun" }
+{ "l_orderkey": 2305, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-02", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-04-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " haggle caref" }
+{ "l_orderkey": 2662, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43090.3, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-12-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". slyly specia" }
+{ "l_orderkey": 2693, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43090.3, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "as are according to th" }
+{ "l_orderkey": 2854, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7014.7, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-08-14", "l_receiptdate": "1994-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the pending" }
+{ "l_orderkey": 2919, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "re slyly. regular ideas detect furiousl" }
+{ "l_orderkey": 2919, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 44092.4, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-02-03", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es doze around the furiously " }
+{ "l_orderkey": 3042, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-02", "l_receiptdate": "1994-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ng the furiously r" }
+{ "l_orderkey": 3527, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 47098.7, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-07-29", "l_receiptdate": "1997-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "unts. express re" }
+{ "l_orderkey": 3844, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5010.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-05-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unwind quickly about the pending, i" }
+{ "l_orderkey": 4007, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5010.5, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y unusual packa" }
+{ "l_orderkey": 4032, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8016.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ometimes even cou" }
+{ "l_orderkey": 4131, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47098.7, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges. ironic pinto be" }
+{ "l_orderkey": 4166, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ily ironic deposits print furiously. iron" }
+{ "l_orderkey": 4322, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10021.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular ideas engage carefully quick" }
+{ "l_orderkey": 4453, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26054.6, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-05-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "express packages are" }
+{ "l_orderkey": 4835, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-01-04", "l_receiptdate": "1995-02-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e carefully regular foxes. deposits are sly" }
+{ "l_orderkey": 4866, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1002.1, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-01", "l_receiptdate": "1997-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "latelets nag. q" }
+{ "l_orderkey": 5030, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22046.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". quickly regular foxes believe" }
+{ "l_orderkey": 5217, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-15", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "pending packages cajole ne" }
+{ "l_orderkey": 5408, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cross the dolphins h" }
+{ "l_orderkey": 5415, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44092.4, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-10-26", "l_receiptdate": "1992-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " requests. unusual theodolites sleep agains" }
+{ "l_orderkey": 5415, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ges around the fur" }
+{ "l_orderkey": 5540, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nic asymptotes could hav" }
+{ "l_orderkey": 5633, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10021.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-03", "l_receiptdate": "1998-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its cajole fluffily fluffily special pinto" }
+{ "l_orderkey": 5696, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-15", "l_receiptdate": "1995-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "n patterns lose slyly fina" }
+{ "l_orderkey": 5762, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27056.7, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-03-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the bold ideas. carefully sp" }
+{ "l_orderkey": 5792, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31065.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-17", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s? furiously even instructions " }
+{ "l_orderkey": 5984, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25052.5, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-07-21", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular accounts. even packages nag slyly" }
+{ "l_orderkey": 68, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30093.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-07-11", "l_receiptdate": "1998-08-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oxes are slyly blithely fin" }
+{ "l_orderkey": 161, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19058.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", regular sheaves sleep along" }
+{ "l_orderkey": 353, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 39120.9, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-15", "l_commitdate": "1994-03-30", "l_receiptdate": "1994-02-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "losely quickly even accounts. c" }
+{ "l_orderkey": 711, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27083.7, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "slyly. ironic asy" }
+{ "l_orderkey": 832, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45139.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "foxes engage slyly alon" }
+{ "l_orderkey": 1504, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " accounts sleep. furiou" }
+{ "l_orderkey": 1956, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16049.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es cajole blithely. pen" }
+{ "l_orderkey": 2017, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49151.9, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-01", "l_receiptdate": "1998-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " after the unusual instructions. sly" }
+{ "l_orderkey": 2209, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10031.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "players. carefully reg" }
+{ "l_orderkey": 2400, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48148.8, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-07", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fore the car" }
+{ "l_orderkey": 2661, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-04-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " foxes affix quickly ironic request" }
+{ "l_orderkey": 3136, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7021.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-09-14", "l_receiptdate": "1994-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic pinto beans are slyly. f" }
+{ "l_orderkey": 3205, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38117.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly quiet accounts. slyly pending pinto " }
+{ "l_orderkey": 3936, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 26080.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "quickly pen" }
+{ "l_orderkey": 4422, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39120.9, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-24", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "en hockey players engage" }
+{ "l_orderkey": 4484, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "the ironic, final theodo" }
+{ "l_orderkey": 4515, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-28", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-04-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ding instructions again" }
+{ "l_orderkey": 4869, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24074.4, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "se deposits above the sly, q" }
+{ "l_orderkey": 4900, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 48148.8, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uickly ironic ideas kindle s" }
+{ "l_orderkey": 4900, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly final acco" }
+{ "l_orderkey": 4931, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8024.8, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dependencies are slyly" }
+{ "l_orderkey": 4932, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15046.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-15", "l_commitdate": "1993-10-25", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly. unusu" }
+{ "l_orderkey": 5377, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " silent wa" }
+{ "l_orderkey": 5445, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-09-15", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "old depend" }
+{ "l_orderkey": 5827, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-16", "l_commitdate": "1998-09-14", "l_receiptdate": "1998-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ans. furiously special instruct" }
+{ "l_orderkey": 71, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 39159.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l accounts sleep across the pack" }
+{ "l_orderkey": 133, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27110.7, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-21", "l_commitdate": "1998-02-23", "l_receiptdate": "1997-12-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yly even gifts after the sl" }
+{ "l_orderkey": 1538, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28114.8, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "bout the fluffily unusual" }
+{ "l_orderkey": 1697, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24098.4, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-29", "l_commitdate": "1996-12-19", "l_receiptdate": "1997-01-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts cajole carefully above the carefully" }
+{ "l_orderkey": 1829, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49200.9, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-26", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ound the quickly " }
+{ "l_orderkey": 1862, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26106.6, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g carefully: thinly ironic deposits af" }
+{ "l_orderkey": 2179, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5020.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-10-08", "l_receiptdate": "1996-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts haggle blithely. ironic, careful theodol" }
+{ "l_orderkey": 2246, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43176.3, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ainst the ironic theodolites haggle fi" }
+{ "l_orderkey": 2469, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 49200.9, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-03-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " requests are car" }
+{ "l_orderkey": 2598, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4016.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " across the furiously fi" }
+{ "l_orderkey": 3010, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9036.9, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-18", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "inal packages. quickly even pinto" }
+{ "l_orderkey": 3047, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17069.7, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-14", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic instruction" }
+{ "l_orderkey": 3488, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48196.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-29", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sly? final requests " }
+{ "l_orderkey": 3748, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "old reques" }
+{ "l_orderkey": 4897, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19077.9, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! ironic, pending dependencies doze furiou" }
+{ "l_orderkey": 5221, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24098.4, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-04", "l_commitdate": "1995-08-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s pinto beans sleep. sly" }
+{ "l_orderkey": 5377, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic, final" }
+{ "l_orderkey": 5409, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38155.8, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-03-29", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic, regular accounts! blithely even" }
+{ "l_orderkey": 5479, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19077.9, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully bo" }
+{ "l_orderkey": 69, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s sleep carefully bold, " }
+{ "l_orderkey": 515, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar deposits th" }
+{ "l_orderkey": 615, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36183.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-06-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. carefully final pinto bea" }
+{ "l_orderkey": 806, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1005.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ar accounts? pending, pending foxes a" }
+{ "l_orderkey": 992, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-15", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nic instructions n" }
+{ "l_orderkey": 1027, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ilent, express foxes near the blithely sp" }
+{ "l_orderkey": 1189, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e regular deposits. quickly quiet deposi" }
+{ "l_orderkey": 1728, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23117.3, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-09-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ns. pending, final ac" }
+{ "l_orderkey": 1859, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12061.2, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es. unusual, silent request" }
+{ "l_orderkey": 1986, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-14", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "yly into the carefully even " }
+{ "l_orderkey": 2082, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12061.2, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-02-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ironic instructions. carefull" }
+{ "l_orderkey": 2086, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 44224.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-30", "l_receiptdate": "1994-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "latelets s" }
+{ "l_orderkey": 2466, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-06-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sly regular deposits. regular, regula" }
+{ "l_orderkey": 2593, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37188.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1993-10-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake bravel" }
+{ "l_orderkey": 2596, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " instructions shall have" }
+{ "l_orderkey": 2756, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31158.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "en instructions use quickly." }
+{ "l_orderkey": 2791, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8040.8, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-24", "l_receiptdate": "1995-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se. close ideas alongs" }
+{ "l_orderkey": 2850, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49249.9, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " slyly unusual req" }
+{ "l_orderkey": 3008, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31158.1, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-01", "l_commitdate": "1996-01-20", "l_receiptdate": "1995-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nts use thinly around the carefully iro" }
+{ "l_orderkey": 3042, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "the requests detect fu" }
+{ "l_orderkey": 3169, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26132.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-04-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ter the regular ideas. slyly iro" }
+{ "l_orderkey": 3335, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13066.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-20", "l_commitdate": "1995-12-20", "l_receiptdate": "1996-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "out the special asymptotes" }
+{ "l_orderkey": 3648, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " above the somas boost furious" }
+{ "l_orderkey": 3778, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 49249.9, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ans. furiously " }
+{ "l_orderkey": 3809, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46234.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l asymptotes. special " }
+{ "l_orderkey": 3845, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "counts do wake blithely. ironic requests " }
+{ "l_orderkey": 3969, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 4020.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "dencies wake blithely? quickly even theodo" }
+{ "l_orderkey": 4288, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39198.9, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-25", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uffy theodolites run" }
+{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19096.9, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nt dependencies. furiously regular ideas d" }
+{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34173.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "pendencies!" }
+{ "l_orderkey": 4900, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40204.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-07-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "luffily final dol" }
+{ "l_orderkey": 5152, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9045.9, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-11", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " cajole furiously alongside of the bo" }
+{ "l_orderkey": 5281, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38193.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-17", "l_commitdate": "1995-12-19", "l_receiptdate": "1996-02-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n asymptotes could wake about th" }
+{ "l_orderkey": 5382, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 48244.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-19", "l_receiptdate": "1992-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nts integrate quickly ca" }
+{ "l_orderkey": 5410, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41209.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-10-20", "l_receiptdate": "1998-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "sly. slyly ironic theodolites" }
+{ "l_orderkey": 5731, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11056.1, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-06", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-06-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously final accounts wake. d" }
+{ "l_orderkey": 197, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1006.1, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-15", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " even, thin dependencies sno" }
+{ "l_orderkey": 229, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 29176.9, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-01-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously pending " }
+{ "l_orderkey": 519, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 19115.9, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "asymptotes. p" }
+{ "l_orderkey": 967, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17103.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic foxes caj" }
+{ "l_orderkey": 1122, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40244.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-07", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-02-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages sleep after the asym" }
+{ "l_orderkey": 1186, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 27164.7, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-08", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "accounts. express, e" }
+{ "l_orderkey": 1283, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1006.1, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-10-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "d the sauternes. slyly ev" }
+{ "l_orderkey": 1317, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r packages impress blithely car" }
+{ "l_orderkey": 1926, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29176.9, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-26", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "es. dependencies according to the fl" }
+{ "l_orderkey": 2501, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33201.3, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "leep furiously packages. even sauternes " }
+{ "l_orderkey": 2598, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12073.2, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits cajol" }
+{ "l_orderkey": 2628, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44268.4, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-14", "l_receiptdate": "1994-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lyly final, pending ide" }
+{ "l_orderkey": 2628, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14085.4, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1993-11-30", "l_receiptdate": "1994-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the furiously unusual pi" }
+{ "l_orderkey": 3014, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36219.6, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-28", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic r" }
+{ "l_orderkey": 3076, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake furiou" }
+{ "l_orderkey": 3457, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages nag furiously against" }
+{ "l_orderkey": 3520, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5030.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-12-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly even ideas haggle " }
+{ "l_orderkey": 3681, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lyly special pinto " }
+{ "l_orderkey": 3717, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 28170.8, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts sleep q" }
+{ "l_orderkey": 4001, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 26158.6, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-26", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate blithely" }
+{ "l_orderkey": 4327, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40244.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-04-17", "l_receiptdate": "1995-06-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. packages are after th" }
+{ "l_orderkey": 4352, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18109.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ding to th" }
+{ "l_orderkey": 4836, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15091.5, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-03-14", "l_receiptdate": "1997-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eep slyly. even requests cajole" }
+{ "l_orderkey": 5095, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2012.2, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "detect car" }
+{ "l_orderkey": 5155, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28170.8, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s cajole. accounts wake. thinly quiet pla" }
+{ "l_orderkey": 5351, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2012.2, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-05-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "g accounts wake furiously slyly even dolph" }
+{ "l_orderkey": 5632, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21128.1, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-03-10", "l_receiptdate": "1996-04-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully regular pinto beans. ironic reques" }
+{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
+{ "l_orderkey": 128, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38269.8, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " cajole careful" }
+{ "l_orderkey": 258, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-02-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully about the fluffily silent dependencies" }
+{ "l_orderkey": 354, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7049.7, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-05-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ously idly ironic accounts-- quickl" }
+{ "l_orderkey": 390, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10071.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. final accounts x-ray beside the" }
+{ "l_orderkey": 450, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5035.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the pinto bea" }
+{ "l_orderkey": 835, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33234.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "onic instructions among the carefully iro" }
+{ "l_orderkey": 960, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1007.1, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-26", "l_receiptdate": "1995-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y ironic packages. quickly even " }
+{ "l_orderkey": 1088, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30213.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "long the packages snooze careful" }
+{ "l_orderkey": 1411, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26184.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c packages. " }
+{ "l_orderkey": 1414, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4028.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle quickly" }
+{ "l_orderkey": 1477, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 32227.2, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-12", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "; quickly regula" }
+{ "l_orderkey": 1509, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17120.7, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously. blithely regular ideas haggle c" }
+{ "l_orderkey": 2150, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29205.9, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully final att" }
+{ "l_orderkey": 3587, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16113.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-06-19", "l_receiptdate": "1996-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y ruthless dolphins to " }
+{ "l_orderkey": 3654, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20142.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s sleep about the slyly " }
+{ "l_orderkey": 3716, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42298.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-03", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " of the pend" }
+{ "l_orderkey": 3814, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15106.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " carefully final deposits haggle slyly" }
+{ "l_orderkey": 3840, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 33234.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-10-06", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "hely silent deposits w" }
+{ "l_orderkey": 3842, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24170.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "phins are quickly" }
+{ "l_orderkey": 4065, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ages haggle carefully" }
+{ "l_orderkey": 4833, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31220.1, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-24", "l_commitdate": "1996-07-15", "l_receiptdate": "1996-07-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ven instructions cajole against the caref" }
+{ "l_orderkey": 5380, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48340.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies haggle car" }
+{ "l_orderkey": 5633, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1007.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-29", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "thely notornis: " }
+{ "l_orderkey": 5729, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39276.9, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-22", "l_commitdate": "1994-11-21", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". special pl" }
+{ "l_orderkey": 5824, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44312.4, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-02-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily fluffily bold" }
+{ "l_orderkey": 5829, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40284.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the carefully ironic accounts. a" }
+{ "l_orderkey": 356, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48388.8, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unusual packages. furiously " }
+{ "l_orderkey": 709, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40324.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ggle fluffily carefully ironic" }
+{ "l_orderkey": 775, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20162.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en dependencies nag slowly " }
+{ "l_orderkey": 871, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1996-01-24", "l_receiptdate": "1996-02-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " haggle furiou" }
+{ "l_orderkey": 965, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20162.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly. carefully pending requ" }
+{ "l_orderkey": 1221, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ajole furiously. blithely expres" }
+{ "l_orderkey": 1351, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25202.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "iously regul" }
+{ "l_orderkey": 1636, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24194.4, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e carefully unusual ideas are f" }
+{ "l_orderkey": 1826, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43348.3, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-08-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss tithes use even ideas. fluffily final t" }
+{ "l_orderkey": 2179, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7056.7, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular dependencies. ironic packages haggle" }
+{ "l_orderkey": 2433, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3024.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "usly pending depos" }
+{ "l_orderkey": 2530, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-02", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial asymptotes snooze slyly regular " }
+{ "l_orderkey": 2560, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slyly final accoun" }
+{ "l_orderkey": 2561, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39315.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-20", "l_commitdate": "1997-12-16", "l_receiptdate": "1998-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests are furiously against the" }
+{ "l_orderkey": 2694, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10081.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-23", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fluffily fluffy accounts. even packages hi" }
+{ "l_orderkey": 3072, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-04-22", "l_receiptdate": "1994-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " theodolites. blithely e" }
+{ "l_orderkey": 3169, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6048.6, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-02-22", "l_receiptdate": "1994-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ular instructions. ca" }
+{ "l_orderkey": 3333, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38307.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts promise bl" }
+{ "l_orderkey": 3586, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-04-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites hagg" }
+{ "l_orderkey": 4354, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1994-12-05", "l_receiptdate": "1995-01-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "efully special packages use fluffily" }
+{ "l_orderkey": 4357, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17137.7, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1997-12-08", "l_receiptdate": "1998-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e carefully furiou" }
+{ "l_orderkey": 4419, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45364.5, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-09-07", "l_receiptdate": "1996-08-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s doze sometimes fluffily regular a" }
+{ "l_orderkey": 4480, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30243.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ven braids us" }
+{ "l_orderkey": 4613, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25202.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y pending platelets x-ray ironically! pend" }
+{ "l_orderkey": 5094, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23186.3, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-19", "l_receiptdate": "1993-07-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "st furiously above the fluffily care" }
+{ "l_orderkey": 5316, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 29234.9, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ckly unusual foxes bo" }
+{ "l_orderkey": 5761, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " pinto beans thrash alongside of the pendi" }
+{ "l_orderkey": 5923, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2016.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "express patterns. even deposits" }
+{ "l_orderkey": 5, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15136.5, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake furiously " }
+{ "l_orderkey": 99, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 36327.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-04", "l_commitdate": "1994-04-17", "l_receiptdate": "1994-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "slyly. slyly e" }
+{ "l_orderkey": 135, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 47427.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-18", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ctions wake slyly abo" }
+{ "l_orderkey": 164, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27245.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ayers wake carefully a" }
+{ "l_orderkey": 224, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " carefully. final platelets " }
+{ "l_orderkey": 449, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4036.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-27", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "are fluffily. requests are furiously" }
+{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic dependencie" }
+{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 25227.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-14", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e packages engag" }
+{ "l_orderkey": 646, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31282.1, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-01-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ag furiousl" }
+{ "l_orderkey": 896, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11100.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "rding to the pinto beans wa" }
+{ "l_orderkey": 1159, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39354.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-10-28", "l_receiptdate": "1992-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " blithely express reques" }
+{ "l_orderkey": 1410, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "to beans b" }
+{ "l_orderkey": 1574, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6054.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "nic, final ideas snooze. " }
+{ "l_orderkey": 1637, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 38345.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "even, pending foxes nod regular" }
+{ "l_orderkey": 1798, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ld packages sleep furiously. depend" }
+{ "l_orderkey": 2276, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 38345.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ans. pinto beans boost c" }
+{ "l_orderkey": 2432, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13118.3, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "arefully about the caref" }
+{ "l_orderkey": 2720, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49445.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-07-14", "l_receiptdate": "1993-07-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " accounts. fluffily bold pack" }
+{ "l_orderkey": 2976, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30273.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c ideas! unusual" }
+{ "l_orderkey": 2980, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sts. slyly regu" }
+{ "l_orderkey": 3108, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-10-01", "l_receiptdate": "1993-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " final requests. " }
+{ "l_orderkey": 3235, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9081.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l courts sleep quickly slyly " }
+{ "l_orderkey": 3264, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24218.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-07", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ctions. quick" }
+{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7063.7, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pending accounts along the" }
+{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42382.2, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously final instruc" }
+{ "l_orderkey": 3492, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " unusual requests. ir" }
+{ "l_orderkey": 3970, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18163.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " maintain slyly. ir" }
+{ "l_orderkey": 4066, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 44400.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express accounts nag bli" }
+{ "l_orderkey": 4545, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8072.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " boost slyly. slyly" }
+{ "l_orderkey": 4929, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26236.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly. fl" }
+{ "l_orderkey": 5088, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10091.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans. special requests af" }
+{ "l_orderkey": 5346, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7063.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "equests use carefully care" }
+{ "l_orderkey": 5634, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16145.6, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ess ideas are carefully pending, even re" }
+{ "l_orderkey": 5636, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15136.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-21", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "efully special" }
+{ "l_orderkey": 5666, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36327.6, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-16", "l_receiptdate": "1994-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "accounts. furiousl" }
+{ "l_orderkey": 98, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1010.11, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-01", "l_commitdate": "1994-12-12", "l_receiptdate": "1994-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". unusual instructions against" }
+{ "l_orderkey": 1059, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26262.86, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ar pinto beans at the furiously " }
+{ "l_orderkey": 1060, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ccounts. foxes maintain care" }
+{ "l_orderkey": 1127, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38384.18, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-07", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ". never final packages boost acro" }
+{ "l_orderkey": 1314, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual accounts slee" }
+{ "l_orderkey": 1477, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8080.88, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-25", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ironic realms wake unusual, even ac" }
+{ "l_orderkey": 1991, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1992-11-29", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ckages? carefully bold depos" }
+{ "l_orderkey": 2343, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old theodolites." }
+{ "l_orderkey": 2470, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "l accounts. deposits nag daringly. express," }
+{ "l_orderkey": 2656, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40404.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-24", "l_receiptdate": "1993-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "refully final pearls. final ideas wake. qu" }
+{ "l_orderkey": 2785, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37374.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-25", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tructions. furiously " }
+{ "l_orderkey": 2850, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30303.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1996-11-29", "l_receiptdate": "1997-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "even ideas. busy pinto beans sleep above t" }
+{ "l_orderkey": 3170, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-01-04", "l_receiptdate": "1998-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". express dolphins use sly" }
+{ "l_orderkey": 3426, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20202.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-24", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sits cajole blit" }
+{ "l_orderkey": 3588, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 37374.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xcuses sleep quickly along th" }
+{ "l_orderkey": 3651, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " sleep blithely furiously do" }
+{ "l_orderkey": 3779, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5050.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-07", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-02-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites. slyly regular a" }
+{ "l_orderkey": 3907, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8080.88, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "furiously final packages." }
+{ "l_orderkey": 3941, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29293.19, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-14", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "g the blithely" }
+{ "l_orderkey": 4033, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans" }
+{ "l_orderkey": 4065, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ncies use furiously. quickly un" }
+{ "l_orderkey": 4068, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ructions. regular, special packag" }
+{ "l_orderkey": 4261, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "into beans " }
+{ "l_orderkey": 4801, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4040.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-23", "l_commitdate": "1996-04-04", "l_receiptdate": "1996-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pitaphs. regular, reg" }
+{ "l_orderkey": 4934, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41414.51, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "wake final, ironic f" }
+{ "l_orderkey": 4995, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48485.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-04", "l_receiptdate": "1996-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nstructions. carefully final depos" }
+{ "l_orderkey": 5189, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4040.44, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". blithely exp" }
+{ "l_orderkey": 5413, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22222.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-10", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits. quick" }
+{ "l_orderkey": 5575, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7070.77, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. final, final " }
+{ "l_orderkey": 192, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15166.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-02-10", "l_receiptdate": "1998-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "he ironic requests haggle about" }
+{ "l_orderkey": 416, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22244.42, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1993-12-17", "l_receiptdate": "1994-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "rint blithely above the pending sentim" }
+{ "l_orderkey": 610, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49544.39, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-29", "l_commitdate": "1995-10-26", "l_receiptdate": "1995-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ular instruc" }
+{ "l_orderkey": 902, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-01", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "into beans thrash blithely about the flu" }
+{ "l_orderkey": 1061, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26288.86, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ave to slee" }
+{ "l_orderkey": 1252, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27299.97, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-22", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "packages hag" }
+{ "l_orderkey": 1575, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39433.29, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-11-05", "l_receiptdate": "1995-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " after the unusual asym" }
+{ "l_orderkey": 1733, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41455.51, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-13", "l_commitdate": "1996-07-08", "l_receiptdate": "1996-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ess notornis. fur" }
+{ "l_orderkey": 1766, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1011.11, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1997-01-07", "l_receiptdate": "1997-02-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly blithely pending accounts. reg" }
+{ "l_orderkey": 2180, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 47522.17, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending, regular ideas. iron" }
+{ "l_orderkey": 2274, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23255.53, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-11-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kly special warhorse" }
+{ "l_orderkey": 3206, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37411.07, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-10-31", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quick theodolites hagg" }
+{ "l_orderkey": 3364, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38422.18, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly express" }
+{ "l_orderkey": 3456, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34377.74, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-29", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usy pinto beans b" }
+{ "l_orderkey": 4002, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eep. quickly" }
+{ "l_orderkey": 4385, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38422.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inal frays. final, bold exc" }
+{ "l_orderkey": 4613, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e blithely against the even, bold pi" }
+{ "l_orderkey": 4674, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " regular requests na" }
+{ "l_orderkey": 4676, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 50555.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r deposits boost boldly quickly quick asymp" }
+{ "l_orderkey": 4705, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22244.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " fluffily pending accounts ca" }
+{ "l_orderkey": 5121, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45499.95, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-09-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial accounts cajole ca" }
+{ "l_orderkey": 5254, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntegrate carefully among the pending" }
+{ "l_orderkey": 5381, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48533.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "luffily spec" }
+{ "l_orderkey": 5413, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 36399.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-12", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular, regular ideas mold! final requests" }
+{ "l_orderkey": 5671, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42466.62, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-17", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "carefully slyly special deposit" }
+{ "l_orderkey": 5731, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6066.66, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-02", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits integrate slyly close platelets. quick" }
+{ "l_orderkey": 289, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6072.66, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "d packages use fluffily furiously" }
+{ "l_orderkey": 481, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31375.41, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-15", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly final packages believe. quick" }
+{ "l_orderkey": 768, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43520.73, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sual ideas wake quickly" }
+{ "l_orderkey": 833, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38460.18, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " platelets promise furiously. " }
+{ "l_orderkey": 1028, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39472.29, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " final dependencies affix a" }
+{ "l_orderkey": 1095, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " bold accounts haggle slyly furiously even" }
+{ "l_orderkey": 1475, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23278.53, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-13", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-03-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "hely regular hocke" }
+{ "l_orderkey": 1698, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19230.09, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-06-21", "l_receiptdate": "1997-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " fluffily e" }
+{ "l_orderkey": 2113, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1998-01-08", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly regular accounts hinder about the" }
+{ "l_orderkey": 2305, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully alongside of " }
+{ "l_orderkey": 2564, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y express requests sleep furi" }
+{ "l_orderkey": 2567, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50605.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully pending foxes are furi" }
+{ "l_orderkey": 2759, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11133.21, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "hely regular " }
+{ "l_orderkey": 2885, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pending packages wake. " }
+{ "l_orderkey": 2887, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fily final packages. regula" }
+{ "l_orderkey": 2982, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21254.31, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-06-08", "l_receiptdate": "1995-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ironic deposits. furiously ex" }
+{ "l_orderkey": 3585, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11133.21, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "even packages" }
+{ "l_orderkey": 3713, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41496.51, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-07-17", "l_receiptdate": "1998-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits wake blithely fina" }
+{ "l_orderkey": 3907, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41496.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-13", "l_commitdate": "1992-10-23", "l_receiptdate": "1992-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages wake along the carefully regul" }
+{ "l_orderkey": 4071, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22266.42, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sits cajole carefully final instructio" }
+{ "l_orderkey": 4769, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15181.65, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular platelets can cajole across the " }
+{ "l_orderkey": 5024, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18217.98, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-02", "l_commitdate": "1997-01-16", "l_receiptdate": "1996-12-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "zle carefully sauternes. quickly" }
+{ "l_orderkey": 5153, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36435.96, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-15", "l_commitdate": "1995-11-08", "l_receiptdate": "1995-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ironic instru" }
+{ "l_orderkey": 5313, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 47569.17, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-12", "l_commitdate": "1997-08-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "pinto beans across the " }
+{ "l_orderkey": 5664, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44532.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-09-26", "l_receiptdate": "1998-10-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ang thinly bold pa" }
+{ "l_orderkey": 5827, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38460.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-18", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly ruthless accounts" }
+{ "l_orderkey": 5888, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing to the spe" }
+{ "l_orderkey": 5955, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "oss the fluffily regular" }
+{ "l_orderkey": 647, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5065.55, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express packages haggle caref" }
+{ "l_orderkey": 964, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1013.11, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-20", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts. quickly even platelets s" }
+{ "l_orderkey": 1027, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20262.2, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-08-29", "l_receiptdate": "1992-06-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar excuses eat f" }
+{ "l_orderkey": 1090, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28367.08, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-20", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s cajole above the regular" }
+{ "l_orderkey": 1188, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9117.99, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-04", "l_commitdate": "1996-06-04", "l_receiptdate": "1996-08-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ow carefully ironic d" }
+{ "l_orderkey": 1347, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28367.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-08-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes after the blithely special i" }
+{ "l_orderkey": 1860, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9117.99, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-03", "l_commitdate": "1996-05-31", "l_receiptdate": "1996-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "c realms print carefully car" }
+{ "l_orderkey": 1892, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48629.28, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-16", "l_commitdate": "1994-06-16", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tornis detect regul" }
+{ "l_orderkey": 2050, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41537.51, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-08-27", "l_receiptdate": "1994-06-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " final theodolites. depende" }
+{ "l_orderkey": 2054, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11144.21, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accou" }
+{ "l_orderkey": 2081, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19249.09, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s affix sometimes express requests. quickly" }
+{ "l_orderkey": 2214, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42550.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ons. deposi" }
+{ "l_orderkey": 2759, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37485.07, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-22", "l_receiptdate": "1994-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lar Tiresias affix ironically carefully sp" }
+{ "l_orderkey": 3207, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2026.22, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "among the ironic, even packages " }
+{ "l_orderkey": 3328, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6078.66, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-01-25", "l_receiptdate": "1993-03-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ffily even instructions detect b" }
+{ "l_orderkey": 3333, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "riously ironic r" }
+{ "l_orderkey": 3363, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20262.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully quiet excuses wake. sl" }
+{ "l_orderkey": 3651, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41537.51, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-10", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely. furiously " }
+{ "l_orderkey": 3750, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47616.17, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-11", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "slowly regular accounts. blithely ev" }
+{ "l_orderkey": 3875, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-13", "l_receiptdate": "1997-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sleep furiously about the deposits. quickl" }
+{ "l_orderkey": 3911, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ss theodolites are blithely along t" }
+{ "l_orderkey": 4160, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25327.75, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-09-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar accounts sleep blithe" }
+{ "l_orderkey": 4263, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47616.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-07-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y. theodolites wake idly ironic do" }
+{ "l_orderkey": 4487, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sual packages should ha" }
+{ "l_orderkey": 4741, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16209.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "final foxes haggle r" }
+{ "l_orderkey": 5411, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-04", "l_receiptdate": "1997-07-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding, special foxes unw" }
+{ "l_orderkey": 5862, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4052.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent deposit" }
+{ "l_orderkey": 5890, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38498.18, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1992-12-09", "l_receiptdate": "1993-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " accounts. carefully final asymptotes" }
+{ "l_orderkey": 355, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31437.41, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y unusual, ironic" }
+{ "l_orderkey": 357, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 26366.86, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-11-26", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " carefully pending accounts use a" }
+{ "l_orderkey": 710, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es. furiously p" }
+{ "l_orderkey": 1253, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19268.09, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-04-22", "l_receiptdate": "1993-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al pinto bea" }
+{ "l_orderkey": 1318, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24338.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ual, unusual packages. fluffy, iro" }
+{ "l_orderkey": 1440, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 46649.06, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely even instructions. " }
+{ "l_orderkey": 1506, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30423.3, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "deposits cajole " }
+{ "l_orderkey": 1604, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 19268.09, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ideas. bol" }
+{ "l_orderkey": 1635, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20282.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "oost according to the carefully even accou" }
+{ "l_orderkey": 1636, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7098.77, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-09-10", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ronic instructions. final" }
+{ "l_orderkey": 1795, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34479.74, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "closely regular instructions wake. " }
+{ "l_orderkey": 2469, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16225.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-19", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-03-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ing asymptotes " }
+{ "l_orderkey": 2532, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9126.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-30", "l_commitdate": "1995-11-23", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "cial ideas haggle slyly pending request" }
+{ "l_orderkey": 2663, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35493.85, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1995-10-16", "l_receiptdate": "1996-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tect. slyly fina" }
+{ "l_orderkey": 2692, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1998-02-11", "l_receiptdate": "1998-03-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "posits. final, express requests nag furi" }
+{ "l_orderkey": 3014, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14197.54, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-19", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". slyly brave platelets nag. careful," }
+{ "l_orderkey": 4004, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39550.29, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts sleep furious" }
+{ "l_orderkey": 4006, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 25352.75, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-02-09", "l_receiptdate": "1995-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests use depos" }
+{ "l_orderkey": 4196, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42592.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " instructions. courts cajole slyly ev" }
+{ "l_orderkey": 4452, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "multipliers x-ray carefully in place of " }
+{ "l_orderkey": 5090, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2028.22, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tes. slowly iro" }
+{ "l_orderkey": 5281, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37522.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ronic dependencies. fluffily final p" }
+{ "l_orderkey": 5345, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 37522.07, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " along the ironically fina" }
+{ "l_orderkey": 5671, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30423.3, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily ironi" }
+{ "l_orderkey": 132, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32483.52, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "d instructions hagg" }
+{ "l_orderkey": 135, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13196.43, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-12-22", "l_receiptdate": "1995-11-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nal ideas. final instr" }
+{ "l_orderkey": 452, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2030.22, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y express instru" }
+{ "l_orderkey": 646, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic packages sleep across th" }
+{ "l_orderkey": 900, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cial pinto beans nag " }
+{ "l_orderkey": 1094, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9135.99, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as. slyly pe" }
+{ "l_orderkey": 1188, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2030.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its breach blit" }
+{ "l_orderkey": 1477, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-31", "l_receiptdate": "1998-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. final pearls kindle. accounts " }
+{ "l_orderkey": 1504, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final theodolites. furiously e" }
+{ "l_orderkey": 1543, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6090.66, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " among the carefully bold or" }
+{ "l_orderkey": 1606, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21317.31, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-02", "l_receiptdate": "1997-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending theodolites prom" }
+{ "l_orderkey": 1829, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 36543.96, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ackages-- express requests sleep; pen" }
+{ "l_orderkey": 1831, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17256.87, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s boost ironic foxe" }
+{ "l_orderkey": 2084, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 15226.65, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tithes. bravely pendi" }
+{ "l_orderkey": 2565, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34513.74, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nstructions was carefu" }
+{ "l_orderkey": 2657, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22332.42, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "r ideas. furiously special dolphins" }
+{ "l_orderkey": 2880, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42634.62, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ions. carefully final accounts are unusual," }
+{ "l_orderkey": 2912, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18271.98, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-13", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts cajole reg" }
+{ "l_orderkey": 3077, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23347.53, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly. fluffily pending dinos across" }
+{ "l_orderkey": 3362, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "packages haggle furi" }
+{ "l_orderkey": 3365, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13196.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-01-31", "l_receiptdate": "1995-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pths wake r" }
+{ "l_orderkey": 3937, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27407.97, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-06", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-02-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven ideas. slyly expr" }
+{ "l_orderkey": 3973, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37559.07, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "inos wake fluffily. pending requests nag " }
+{ "l_orderkey": 4101, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22332.42, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly express instructions. careful" }
+{ "l_orderkey": 4644, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-02-28", "l_receiptdate": "1998-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits according to the" }
+{ "l_orderkey": 4930, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20302.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he carefully" }
+{ "l_orderkey": 5093, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30453.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ithely ironic sheaves use fluff" }
+{ "l_orderkey": 5191, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests! ironic theodolites cajole care" }
+{ "l_orderkey": 5317, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ts about the packages cajole furio" }
+{ "l_orderkey": 5440, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3045.33, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y. accounts haggle along the blit" }
+{ "l_orderkey": 5794, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14211.54, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uriously carefully ironic reque" }
+{ "l_orderkey": 5798, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 32483.52, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-05-03", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ubt blithely above the " }
+{ "l_orderkey": 5921, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5075.55, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eas cajole across the final, fi" }
+{ "l_orderkey": 5927, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8120.88, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-24", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ilent dependencies nod c" }
+{ "l_orderkey": 66, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31499.41, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ut the unusual accounts sleep at the bo" }
+{ "l_orderkey": 69, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48773.28, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-09-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "regular epitaphs. carefully even ideas hag" }
+{ "l_orderkey": 100, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22354.42, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nto beans alongside of the fi" }
+{ "l_orderkey": 130, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13209.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending dolphins sleep furious" }
+{ "l_orderkey": 514, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43692.73, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular " }
+{ "l_orderkey": 1861, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 38612.18, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pending deposits cajole quic" }
+{ "l_orderkey": 1925, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e carefully regul" }
+{ "l_orderkey": 2148, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21338.31, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-28", "l_commitdate": "1995-05-26", "l_receiptdate": "1995-06-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "deposits ag" }
+{ "l_orderkey": 2241, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 22354.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", ironic depen" }
+{ "l_orderkey": 2437, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular deposits. ironic fray" }
+{ "l_orderkey": 2534, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-29", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sual depos" }
+{ "l_orderkey": 2755, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48773.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-10", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "yly even epitaphs for the " }
+{ "l_orderkey": 3136, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26418.86, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-11-07", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eep fluffily. daringly silent attainments d" }
+{ "l_orderkey": 3200, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17273.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-06", "l_commitdate": "1996-04-21", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "side of the furiously pendin" }
+{ "l_orderkey": 3619, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27434.97, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pecial accounts haggle care" }
+{ "l_orderkey": 3682, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18289.98, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-05-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "regular dependencies" }
+{ "l_orderkey": 3905, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ully furiously furious packag" }
+{ "l_orderkey": 4007, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41660.51, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eposits. regular epitaphs boost blithely." }
+{ "l_orderkey": 4421, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18289.98, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". regular, s" }
+{ "l_orderkey": 4547, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "slyly express a" }
+{ "l_orderkey": 4706, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5080.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ptotes haggle ca" }
+{ "l_orderkey": 4711, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 45724.95, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-14", "l_receiptdate": "1998-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic theodolites " }
+{ "l_orderkey": 4900, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "heodolites. request" }
+{ "l_orderkey": 4901, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ect across the furiou" }
+{ "l_orderkey": 5603, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49789.39, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully silent requests. carefully fin" }
+{ "l_orderkey": 288, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49838.39, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ic excuses sleep always spe" }
+{ "l_orderkey": 353, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ual accounts! carefu" }
+{ "l_orderkey": 705, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35598.85, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "carefully ironic accounts" }
+{ "l_orderkey": 807, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49838.39, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-13", "l_receiptdate": "1993-12-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " furiously according to the un" }
+{ "l_orderkey": 960, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-01", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts. fluffily regular requests " }
+{ "l_orderkey": 966, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42718.62, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tions boost furiously car" }
+{ "l_orderkey": 1508, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 38650.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-30", "l_commitdate": "1998-06-23", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tes wake furiously regular w" }
+{ "l_orderkey": 1575, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 48821.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-10-25", "l_receiptdate": "1995-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "cies. regu" }
+{ "l_orderkey": 1794, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23393.53, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-11-30", "l_receiptdate": "1997-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "usly unusual theodolites doze about " }
+{ "l_orderkey": 1794, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 47804.17, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1997-11-30", "l_receiptdate": "1998-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " haggle slyly. furiously express orbit" }
+{ "l_orderkey": 1856, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20342.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-05-06", "l_receiptdate": "1992-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ost carefully. slyly bold accounts" }
+{ "l_orderkey": 2339, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13222.43, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-10", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ges. blithely special depend" }
+{ "l_orderkey": 2342, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24410.64, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-07-22", "l_receiptdate": "1996-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nstructions c" }
+{ "l_orderkey": 2436, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18307.98, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y ironic accounts. furiously even packa" }
+{ "l_orderkey": 2790, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50855.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fter the regular ideas. f" }
+{ "l_orderkey": 3079, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38650.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-25", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully regular realms" }
+{ "l_orderkey": 3236, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10171.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "arefully. fluffily reg" }
+{ "l_orderkey": 3270, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual packages" }
+{ "l_orderkey": 3360, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29496.19, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "hely gifts. spe" }
+{ "l_orderkey": 3393, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16273.76, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uses. instructions after the blithely " }
+{ "l_orderkey": 3494, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40684.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lites haggle furiously about the fin" }
+{ "l_orderkey": 3526, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23393.53, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "special, regular packages cajole. " }
+{ "l_orderkey": 3617, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46787.06, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-06-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar theodolites. regu" }
+{ "l_orderkey": 3623, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33564.63, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-13", "l_receiptdate": "1997-04-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "odolites. blithely spe" }
+{ "l_orderkey": 3648, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s requests. silent asymp" }
+{ "l_orderkey": 3686, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7119.77, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-02", "l_receiptdate": "1998-07-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ake carefully carefully q" }
+{ "l_orderkey": 3782, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34581.74, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gage after the even" }
+{ "l_orderkey": 4193, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3051.33, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-29", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "osits above the depo" }
+{ "l_orderkey": 4545, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27461.97, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-02-22", "l_receiptdate": "1993-03-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ccounts haggle carefully. deposits " }
+{ "l_orderkey": 4678, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18307.98, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usly ironic " }
+{ "l_orderkey": 4833, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11188.21, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s nag above the busily sile" }
+{ "l_orderkey": 5156, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21359.31, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1997-01-30", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts detect against the furiously reg" }
+{ "l_orderkey": 5410, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48821.28, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-09-11", "l_receiptdate": "1998-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " about the slyly even courts. quickly regul" }
+{ "l_orderkey": 5508, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4068.44, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-09-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fluffily about the even " }
+{ "l_orderkey": 5920, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2034.22, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " evenly spe" }
+{ "l_orderkey": 133, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29525.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-28", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-03-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " the carefully regular theodoli" }
+{ "l_orderkey": 226, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2036.22, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "al platelets. express somas " }
+{ "l_orderkey": 226, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ep carefully regular accounts. ironic" }
+{ "l_orderkey": 293, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13235.43, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-17", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " wake after the quickly even deposits. bli" }
+{ "l_orderkey": 454, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24434.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-26", "l_commitdate": "1996-03-23", "l_receiptdate": "1996-05-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "le. deposits after the ideas nag unusual pa" }
+{ "l_orderkey": 514, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34615.74, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ily even patterns. bold, silent instruc" }
+{ "l_orderkey": 610, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26470.86, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cross the furiously even theodolites sl" }
+{ "l_orderkey": 902, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8144.88, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " orbits al" }
+{ "l_orderkey": 934, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y unusual requests dazzle above t" }
+{ "l_orderkey": 961, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7126.77, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-23", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "usual dolphins. ironic pearls sleep blit" }
+{ "l_orderkey": 999, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15271.65, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-12", "l_commitdate": "1993-10-18", "l_receiptdate": "1994-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y ironic requests. carefully regu" }
+{ "l_orderkey": 1349, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45814.95, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-24", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic, unusual deposits wake carefu" }
+{ "l_orderkey": 1379, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50905.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "olphins. ca" }
+{ "l_orderkey": 1441, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "special requests ha" }
+{ "l_orderkey": 1475, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al deposits use. ironic packages along the " }
+{ "l_orderkey": 1664, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " use. ironic deposits integrate. slyly unu" }
+{ "l_orderkey": 1734, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-07-17", "l_receiptdate": "1994-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "final warhorses." }
+{ "l_orderkey": 1793, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests nod ac" }
+{ "l_orderkey": 2213, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20362.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-21", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "iously express accounts; " }
+{ "l_orderkey": 2308, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24434.64, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1992-12-24", "l_receiptdate": "1993-03-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts sleep. busy excuses along the s" }
+{ "l_orderkey": 2374, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41742.51, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "heodolites. requests" }
+{ "l_orderkey": 2629, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6108.66, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-05-29", "l_receiptdate": "1998-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dolites hinder bli" }
+{ "l_orderkey": 2631, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15271.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-30", "l_commitdate": "1993-11-06", "l_receiptdate": "1993-10-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y. furiously even pinto be" }
+{ "l_orderkey": 2725, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23416.53, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular deposits. brave foxes " }
+{ "l_orderkey": 2756, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits grow bold sheaves; iro" }
+{ "l_orderkey": 2948, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-09-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unusual excuses use about the " }
+{ "l_orderkey": 3201, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27488.97, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-08-24", "l_receiptdate": "1993-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits are slyly along" }
+{ "l_orderkey": 3236, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7126.77, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-27", "l_commitdate": "1996-12-18", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dolites. slyly unus" }
+{ "l_orderkey": 3428, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-01", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly regular pinto beans sleep" }
+{ "l_orderkey": 3747, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23416.53, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-12-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages are ironic" }
+{ "l_orderkey": 4035, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1018.11, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. quickly " }
+{ "l_orderkey": 4386, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28507.08, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-03-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". quick packages play slyly " }
+{ "l_orderkey": 5188, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p according to the sometimes regu" }
+{ "l_orderkey": 5282, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36651.96, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-20", "l_commitdate": "1998-04-10", "l_receiptdate": "1998-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "re slyly accor" }
+{ "l_orderkey": 5314, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10181.1, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "latelets haggle final" }
+{ "l_orderkey": 5408, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-10-17", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "thely ironic requests alongside of the sl" }
+{ "l_orderkey": 5766, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-10-30", "l_receiptdate": "1993-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly even requests. furiou" }
+{ "l_orderkey": 5857, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12217.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-02-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts. express, final" }
+{ "l_orderkey": 101, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49936.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-27", "l_receiptdate": "1996-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts-- final packages sleep furiousl" }
+{ "l_orderkey": 256, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40764.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-30", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-12-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nal theodolites. deposits cajole s" }
+{ "l_orderkey": 261, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49936.39, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-12", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ions. bold accounts " }
+{ "l_orderkey": 356, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. unusual, final" }
+{ "l_orderkey": 1061, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2038.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-15", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". regular accounts impre" }
+{ "l_orderkey": 1444, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6114.66, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al accounts. br" }
+{ "l_orderkey": 1607, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37707.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-02-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "alongside " }
+{ "l_orderkey": 1857, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8152.88, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "slyly about the fluffily silent req" }
+{ "l_orderkey": 1957, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-08-31", "l_receiptdate": "1998-08-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "express packages maintain fluffi" }
+{ "l_orderkey": 2276, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5095.55, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-18", "l_receiptdate": "1996-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ias instea" }
+{ "l_orderkey": 2279, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 32611.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-20", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "re quickly. furiously ironic ide" }
+{ "l_orderkey": 2306, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21401.31, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-07", "l_commitdate": "1995-09-18", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " ironic pinto " }
+{ "l_orderkey": 2499, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12229.32, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-18", "l_commitdate": "1995-12-13", "l_receiptdate": "1995-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously along the r" }
+{ "l_orderkey": 2563, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1993-12-31", "l_receiptdate": "1994-02-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lent requests should integrate; carefully e" }
+{ "l_orderkey": 2659, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2038.22, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts above the fluffily express fo" }
+{ "l_orderkey": 3201, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50955.5, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " deposits. express, ir" }
+{ "l_orderkey": 3427, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-12", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are carefull" }
+{ "l_orderkey": 3590, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve furiously final instructions. slyly regu" }
+{ "l_orderkey": 3687, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-06-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "foxes cajole quickly about the furiously f" }
+{ "l_orderkey": 3911, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14267.54, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-28", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e blithely brave depo" }
+{ "l_orderkey": 4294, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37707.07, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-09-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cial packages nag f" }
+{ "l_orderkey": 4613, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-06-01", "l_receiptdate": "1998-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously express" }
+{ "l_orderkey": 4675, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1019.11, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-04-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "unts. caref" }
+{ "l_orderkey": 4775, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-10-12", "l_receiptdate": "1995-09-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "eep never with the slyly regular acc" }
+{ "l_orderkey": 5158, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20382.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-03", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "latelets use accordin" }
+{ "l_orderkey": 5219, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20382.2, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-05-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e along the ironic," }
+{ "l_orderkey": 5478, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 25477.75, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-08", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unusual, pending requests haggle accoun" }
+{ "l_orderkey": 5574, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27515.97, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ecial realms. furiously entici" }
+{ "l_orderkey": 5923, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42802.62, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y regular theodolites w" }
+{ "l_orderkey": 5959, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ely silent deposits. " }
+{ "l_orderkey": 35, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 34684.08, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-08", "l_commitdate": "1996-01-15", "l_receiptdate": "1995-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". silent, unusual deposits boost" }
+{ "l_orderkey": 36, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 42845.04, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " careful courts. special " }
+{ "l_orderkey": 97, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13261.56, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-04-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ayers cajole against the furiously" }
+{ "l_orderkey": 132, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43865.16, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-09-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y pending theodolites" }
+{ "l_orderkey": 353, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41824.92, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "refully final theodoli" }
+{ "l_orderkey": 583, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 47945.64, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nts are fluffily. furiously even re" }
+{ "l_orderkey": 611, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39784.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-10", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the evenly bold requests. furious" }
+{ "l_orderkey": 774, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 2040.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1996-02-10", "l_receiptdate": "1995-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "accounts; slyly regular" }
+{ "l_orderkey": 899, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47945.64, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-05-30", "l_receiptdate": "1998-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "furiously final foxes after the s" }
+{ "l_orderkey": 1221, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41824.92, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-07-02", "l_receiptdate": "1992-05-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ns. bold deposit" }
+{ "l_orderkey": 1477, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 33663.96, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-12", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "yly regular p" }
+{ "l_orderkey": 1505, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4080.48, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "side of the s" }
+{ "l_orderkey": 1733, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29583.48, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-07-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ns detect among the special accounts. qu" }
+{ "l_orderkey": 1824, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45905.4, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ent Tiresias. quickly express " }
+{ "l_orderkey": 1830, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ely even a" }
+{ "l_orderkey": 1959, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15301.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " quickly sp" }
+{ "l_orderkey": 2054, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31623.72, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se bold, regular accounts. unusual depos" }
+{ "l_orderkey": 2720, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 51006.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-07-29", "l_receiptdate": "1993-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l requests. deposits nag furiously" }
+{ "l_orderkey": 2944, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44885.28, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1997-10-28", "l_receiptdate": "1998-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ickly special theodolit" }
+{ "l_orderkey": 3013, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35704.2, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-05-04", "l_receiptdate": "1997-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely accord" }
+{ "l_orderkey": 3142, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15301.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-15", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "instructions are. ironic packages doz" }
+{ "l_orderkey": 3174, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-07", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic deposits among t" }
+{ "l_orderkey": 3175, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28563.36, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-10-05", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ore the even, silent foxes. b" }
+{ "l_orderkey": 3367, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "even packages sleep blithely slyly expr" }
+{ "l_orderkey": 3425, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11221.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckly final deposits use quickly?" }
+{ "l_orderkey": 3619, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "theodolites detect abo" }
+{ "l_orderkey": 4135, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32643.84, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " ideas. requests use. furiously" }
+{ "l_orderkey": 4164, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9181.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-08-13", "l_receiptdate": "1998-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "re fluffily slyly bold requests. " }
+{ "l_orderkey": 4292, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35704.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-23", "l_commitdate": "1992-04-04", "l_receiptdate": "1992-04-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dugouts use. furiously bold packag" }
+{ "l_orderkey": 4579, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-15", "l_receiptdate": "1995-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully perman" }
+{ "l_orderkey": 4901, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16321.92, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-18", "l_receiptdate": "1998-04-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "deposits. blithely fin" }
+{ "l_orderkey": 4903, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27543.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans are; " }
+{ "l_orderkey": 4960, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14281.68, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-04-17", "l_receiptdate": "1995-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "accounts. warhorses are. grouches " }
+{ "l_orderkey": 5313, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 21422.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-09-02", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he blithely regular packages. quickly" }
+{ "l_orderkey": 5317, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 37744.44, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-10-24", "l_receiptdate": "1994-09-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "totes nag theodolites. pend" }
+{ "l_orderkey": 5671, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25503.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-03-28", "l_receiptdate": "1998-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "cording to the quickly final requests-- " }
+{ "l_orderkey": 35, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7147.84, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-01-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the carefully regular " }
+{ "l_orderkey": 163, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13274.56, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inal requests. even pinto beans hag" }
+{ "l_orderkey": 1056, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37781.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " special packages. qui" }
+{ "l_orderkey": 1060, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 36760.32, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-04-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r the quickly" }
+{ "l_orderkey": 1377, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39823.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e ironic, regular requests. carefully " }
+{ "l_orderkey": 1410, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15316.8, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-06-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold packages are fluf" }
+{ "l_orderkey": 1764, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y quickly regular packages. car" }
+{ "l_orderkey": 1825, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23485.76, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " wake express, even r" }
+{ "l_orderkey": 1890, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 10211.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". even, unusual inst" }
+{ "l_orderkey": 2053, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ts. fluffily final mul" }
+{ "l_orderkey": 2054, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32675.84, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages thrash. carefully final" }
+{ "l_orderkey": 2433, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43908.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ular requests. slyly even pa" }
+{ "l_orderkey": 2435, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8168.96, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-03", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ng the fluffily special foxes nag " }
+{ "l_orderkey": 2469, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30633.6, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-21", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " accounts. regular theodolites affix fu" }
+{ "l_orderkey": 2563, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5105.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the quickly final theodolite" }
+{ "l_orderkey": 2720, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27570.24, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-08-06", "l_receiptdate": "1993-07-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "eas. carefully regular " }
+{ "l_orderkey": 2758, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ptotes sleep furiously" }
+{ "l_orderkey": 2816, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4084.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests print above the final deposits" }
+{ "l_orderkey": 2818, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12253.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lms. quickly bold asymp" }
+{ "l_orderkey": 2914, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9190.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-17", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. carefully final foxes ar" }
+{ "l_orderkey": 2919, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50034.88, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1994-02-28", "l_receiptdate": "1993-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "hely final inst" }
+{ "l_orderkey": 4004, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23485.76, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " bold theodolites? special packages accordi" }
+{ "l_orderkey": 4134, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34718.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e furiously regular sheaves sleep" }
+{ "l_orderkey": 4192, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15316.8, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e slyly special grouches. express pinto b" }
+{ "l_orderkey": 4197, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37781.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully enticing decoys boo" }
+{ "l_orderkey": 4231, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ublate. theodoli" }
+{ "l_orderkey": 4386, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21443.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e pending, sp" }
+{ "l_orderkey": 4903, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1021.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-23", "l_commitdate": "1992-06-13", "l_receiptdate": "1992-05-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nusual requests" }
+{ "l_orderkey": 5093, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-11-14", "l_receiptdate": "1994-01-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he final foxes. fluffily ironic " }
+{ "l_orderkey": 5346, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25528.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "he ironic ideas are boldly slyly ironi" }
+{ "l_orderkey": 5477, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19401.28, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-03", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-03-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ost carefully packages." }
+{ "l_orderkey": 5538, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4084.48, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-02-17", "l_receiptdate": "1994-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ithely along the c" }
+{ "l_orderkey": 5763, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 47992.64, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gle slyly. slyly final re" }
+{ "l_orderkey": 5858, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uffily unusual pinto beans sleep" }
+{ "l_orderkey": 391, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " escapades sleep furiously about " }
+{ "l_orderkey": 482, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1022.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-20", "l_receiptdate": "1996-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es. quickly ironic escapades sleep furious" }
+{ "l_orderkey": 513, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44973.28, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kages sleep boldly ironic theodolites. acco" }
+{ "l_orderkey": 518, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47017.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ". blithely even ideas cajole furiously. b" }
+{ "l_orderkey": 708, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33729.96, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s boost carefully ruthless theodolites. f" }
+{ "l_orderkey": 801, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al accounts. carefully regular foxes wake" }
+{ "l_orderkey": 868, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43951.16, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly silent deposits wake dar" }
+{ "l_orderkey": 1125, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26575.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l instruction" }
+{ "l_orderkey": 1248, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20442.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-12", "l_commitdate": "1992-03-23", "l_receiptdate": "1992-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal foxes cajole carefully slyl" }
+{ "l_orderkey": 1670, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "fily special ideas " }
+{ "l_orderkey": 1890, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 45995.4, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-15", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he carefully regular sauternes. ironic fret" }
+{ "l_orderkey": 2016, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8176.96, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "mptotes haggle ideas. packages wake flu" }
+{ "l_orderkey": 2177, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11243.32, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-20", "l_commitdate": "1997-03-07", "l_receiptdate": "1997-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gainst the ca" }
+{ "l_orderkey": 2311, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50083.88, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas sleep" }
+{ "l_orderkey": 2372, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e carefully blithely even epitaphs. r" }
+{ "l_orderkey": 2631, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 42929.04, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-12-01", "l_receiptdate": "1994-01-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ect carefully at the furiously final the" }
+{ "l_orderkey": 2912, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8176.96, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hs cajole over the slyl" }
+{ "l_orderkey": 3234, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 51106.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly regular ideas according to the regula" }
+{ "l_orderkey": 3236, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " final pinto " }
+{ "l_orderkey": 3395, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39862.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-01-17", "l_receiptdate": "1994-12-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "riously unusual theodolites. fur" }
+{ "l_orderkey": 3491, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " grow against the boldly pending pinto bea" }
+{ "l_orderkey": 3492, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 48039.64, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-03-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deposits. quickly express " }
+{ "l_orderkey": 3585, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ounts use. express, final platelets us" }
+{ "l_orderkey": 3649, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lly bold requests nag; " }
+{ "l_orderkey": 3686, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " furiously unusual accou" }
+{ "l_orderkey": 3842, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-02", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r pinto be" }
+{ "l_orderkey": 4039, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17376.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-20", "l_receiptdate": "1998-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " regular foxes haggle carefully bo" }
+{ "l_orderkey": 4160, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y bold package" }
+{ "l_orderkey": 4161, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic dolphins. in" }
+{ "l_orderkey": 4355, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-28", "l_receiptdate": "1997-02-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts affix ironic" }
+{ "l_orderkey": 4387, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " boost slyly ironic instructions. furiou" }
+{ "l_orderkey": 4484, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41906.92, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ress accounts. ironic deposits unwind fur" }
+{ "l_orderkey": 4583, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-08", "l_receiptdate": "1994-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "detect. doggedly regular pi" }
+{ "l_orderkey": 4676, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 29641.48, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-29", "l_commitdate": "1995-11-12", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly regular theodolites sleep." }
+{ "l_orderkey": 4706, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23508.76, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deas across t" }
+{ "l_orderkey": 4807, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9199.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-03-01", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "may are blithely. carefully even pinto b" }
+{ "l_orderkey": 4838, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-23", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly blithely unusual foxes. even package" }
+{ "l_orderkey": 4868, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-26", "l_commitdate": "1997-05-16", "l_receiptdate": "1997-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "osits. final foxes boost regular," }
+{ "l_orderkey": 4869, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30663.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1994-11-20", "l_receiptdate": "1995-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gedly even requests. s" }
+{ "l_orderkey": 5350, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19420.28, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-11-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "romise slyly alongsi" }
+{ "l_orderkey": 5511, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lphins. carefully blithe de" }
+{ "l_orderkey": 5664, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25553.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-29", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eposits: furiously ironic grouch" }
+{ "l_orderkey": 5666, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-05-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ideas. regular packag" }
+{ "l_orderkey": 5895, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48039.64, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r packages wake carefull" }
+{ "l_orderkey": 291, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21485.52, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-05-10", "l_receiptdate": "1994-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y quickly regular theodolites. final t" }
+{ "l_orderkey": 1059, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly regular theodo" }
+{ "l_orderkey": 1474, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30693.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly. evenly express " }
+{ "l_orderkey": 1505, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51156.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly special platelets. requests ar" }
+{ "l_orderkey": 1607, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-02-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uches cajole. accounts ar" }
+{ "l_orderkey": 2113, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40924.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1997-12-11", "l_receiptdate": "1998-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "bout the quickly ironic t" }
+{ "l_orderkey": 2242, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15346.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "its. carefully express packages cajole. bli" }
+{ "l_orderkey": 2503, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33762.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nal courts integrate according to the" }
+{ "l_orderkey": 2823, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20462.4, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-12-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "its sleep between the unusual, ironic pac" }
+{ "l_orderkey": 2913, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-09-27", "l_receiptdate": "1997-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". final packages a" }
+{ "l_orderkey": 3011, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42971.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "osits haggle quickly pending, " }
+{ "l_orderkey": 3329, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1023.12, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-22", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "regular packages are carefull" }
+{ "l_orderkey": 3392, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1996-01-17", "l_receiptdate": "1995-12-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the fluffily bold deposits." }
+{ "l_orderkey": 3586, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 33762.96, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "iously regular pinto beans integrate" }
+{ "l_orderkey": 3813, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-30", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y ideas. final ideas about the sp" }
+{ "l_orderkey": 3877, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7161.84, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-14", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar dolphins cajole silently " }
+{ "l_orderkey": 3941, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits haggle furiously even" }
+{ "l_orderkey": 4065, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 29670.48, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-08-19", "l_receiptdate": "1994-07-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "equests. packages sleep slyl" }
+{ "l_orderkey": 4455, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34786.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " slyly ironic requests. quickly even d" }
+{ "l_orderkey": 4548, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 48086.64, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-24", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. excuses use slyly spec" }
+{ "l_orderkey": 4673, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9208.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ages nag across " }
+{ "l_orderkey": 4967, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1023.12, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits. unusual frets thrash furiously" }
+{ "l_orderkey": 5024, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42971.04, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-02", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate. busily spec" }
+{ "l_orderkey": 5059, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " special ideas poach blithely qu" }
+{ "l_orderkey": 5095, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28647.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-06-27", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " into the final courts. ca" }
+{ "l_orderkey": 5189, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14323.68, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-23", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unusual packag" }
+{ "l_orderkey": 5414, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49109.76, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " silent dolphins; fluffily regular tithe" }
+{ "l_orderkey": 5700, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30693.6, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly blithely final instructions. fl" }
+{ "l_orderkey": 5760, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits nag. even, regular ideas cajole b" }
+{ "l_orderkey": 5763, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8184.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-23", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "foxes wake slyly. car" }
+{ "l_orderkey": 5, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26627.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sts use slyly quickly special instruc" }
+{ "l_orderkey": 96, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23554.76, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep-- carefully reg" }
+{ "l_orderkey": 99, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5120.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests cajole fluffily waters. blithe" }
+{ "l_orderkey": 290, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23554.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-02-21", "l_receiptdate": "1994-04-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully unusual packages. " }
+{ "l_orderkey": 390, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17410.04, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ending, pending pinto beans wake slyl" }
+{ "l_orderkey": 420, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 40964.8, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " after the special" }
+{ "l_orderkey": 708, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3072.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly pending foxes. " }
+{ "l_orderkey": 1088, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3072.36, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-08-02", "l_receiptdate": "1992-06-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "pecial theodolites " }
+{ "l_orderkey": 1283, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44037.16, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "requests sleep slyly about the " }
+{ "l_orderkey": 1346, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6144.72, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-13", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-09-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "inst the furiously final theodolites. caref" }
+{ "l_orderkey": 1378, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18434.16, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-05-16", "l_receiptdate": "1996-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " theodolites. i" }
+{ "l_orderkey": 1671, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11265.32, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-18", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes sleep blithely" }
+{ "l_orderkey": 1697, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27651.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly regular packages across the silent, b" }
+{ "l_orderkey": 1703, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 49157.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ggle slyly furiously regular theodol" }
+{ "l_orderkey": 1731, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 41988.92, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-05", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle across the blithely ironi" }
+{ "l_orderkey": 1985, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 43013.04, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " patterns? final requests after the sp" }
+{ "l_orderkey": 2209, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24578.88, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-09", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " along the bol" }
+{ "l_orderkey": 2306, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uld have to mold. s" }
+{ "l_orderkey": 2465, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20482.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uriously? furiously ironic excu" }
+{ "l_orderkey": 2594, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13313.56, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully special accounts use courts" }
+{ "l_orderkey": 2629, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31747.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-24", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ate blithely bold, regular deposits. bold" }
+{ "l_orderkey": 2722, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21506.52, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully around the furiously ironic pac" }
+{ "l_orderkey": 3392, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7168.84, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-07", "l_commitdate": "1996-01-09", "l_receiptdate": "1995-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "as. express, final accounts dou" }
+{ "l_orderkey": 3587, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31747.72, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "press fluffily regul" }
+{ "l_orderkey": 5089, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47109.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "above the express accounts. exc" }
+{ "l_orderkey": 5159, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39940.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-08", "l_receiptdate": "1997-01-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re furiously after the pending dolphin" }
+{ "l_orderkey": 5184, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19458.28, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-15", "l_commitdate": "1998-10-12", "l_receiptdate": "1998-11-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "refully express platelets sleep carefull" }
+{ "l_orderkey": 5223, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25603.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y express ideas impress" }
+{ "l_orderkey": 5347, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48133.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-03-29", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "across the slyly bol" }
+{ "l_orderkey": 5696, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19458.28, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-31", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "unusual requests sleep furiously ru" }
+{ "l_orderkey": 5765, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nic requests. deposits wake quickly among " }
+{ "l_orderkey": 5798, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14337.68, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he special, bold packages. carefully iron" }
+{ "l_orderkey": 356, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37929.44, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-08-24", "l_receiptdate": "1994-08-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ndencies are since the packag" }
+{ "l_orderkey": 390, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11276.32, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "according to the foxes are furiously " }
+{ "l_orderkey": 868, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19477.28, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ely even deposits lose blithe" }
+{ "l_orderkey": 1346, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49205.76, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " along the carefully spec" }
+{ "l_orderkey": 1477, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43055.04, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-02", "l_commitdate": "1997-11-02", "l_receiptdate": "1997-11-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lithely after the ir" }
+{ "l_orderkey": 1795, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 32803.84, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " asymptotes across the bold," }
+{ "l_orderkey": 2690, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46130.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-06-02", "l_receiptdate": "1996-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ounts. slyly regular dependencies wa" }
+{ "l_orderkey": 2848, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34854.08, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-15", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ts along the blithely regu" }
+{ "l_orderkey": 2883, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27678.24, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. brave pinto beans nag furiously" }
+{ "l_orderkey": 3264, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11276.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "regular packages" }
+{ "l_orderkey": 3393, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45105.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ld requests hag" }
+{ "l_orderkey": 4069, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3075.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake furiously! slyl" }
+{ "l_orderkey": 4131, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30753.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-04-13", "l_receiptdate": "1998-04-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he fluffily express depen" }
+{ "l_orderkey": 4230, "l_partkey": 125, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 51256.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. final instructions in" }
+{ "l_orderkey": 4288, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7175.84, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-15", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ngside of the special platelet" }
+{ "l_orderkey": 4291, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44080.16, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. quietly regular " }
+{ "l_orderkey": 5124, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45105.28, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-13", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. carefully unusual d" }
+{ "l_orderkey": 5218, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33828.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-09-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ronic instructi" }
+{ "l_orderkey": 5314, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16401.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "hely unusual packages acc" }
+{ "l_orderkey": 5925, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31778.72, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-02", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly. furiously regular deposi" }
+{ "l_orderkey": 164, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38992.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-04", "l_commitdate": "1992-11-23", "l_receiptdate": "1993-01-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts cajole fluffily regular packages. b" }
+{ "l_orderkey": 448, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts thrash quickly among the b" }
+{ "l_orderkey": 641, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18470.16, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p blithely bold packages. quick" }
+{ "l_orderkey": 804, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ehind the quietly regular pac" }
+{ "l_orderkey": 897, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13339.56, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-17", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "bold accounts mold carefully! braids" }
+{ "l_orderkey": 1024, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "des the slyly even" }
+{ "l_orderkey": 1027, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2052.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-09-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. quickly unusual waters inside " }
+{ "l_orderkey": 1184, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3078.36, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1997-12-19", "l_receiptdate": "1998-02-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ar packages. final packages cajol" }
+{ "l_orderkey": 1728, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1026.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly. carefully ex" }
+{ "l_orderkey": 1793, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nic foxes along the even" }
+{ "l_orderkey": 2049, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-02-04", "l_receiptdate": "1995-12-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ial accounts are among the furiously perma" }
+{ "l_orderkey": 2375, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20522.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ckages! blithely enticing deposi" }
+{ "l_orderkey": 2533, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38992.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "of the regular accounts. even packages caj" }
+{ "l_orderkey": 2594, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24626.88, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar accounts sleep fur" }
+{ "l_orderkey": 2752, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22574.64, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-20", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "equests nag. regular dependencies are furio" }
+{ "l_orderkey": 2820, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33861.96, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "carefully even pinto beans. " }
+{ "l_orderkey": 3040, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-06", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-08-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "x furiously bold packages. expres" }
+{ "l_orderkey": 3365, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24626.88, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-01-09", "l_receiptdate": "1995-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans? carefully regula" }
+{ "l_orderkey": 3396, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27705.24, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-07-26", "l_receiptdate": "1994-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " theodolites " }
+{ "l_orderkey": 3492, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7182.84, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-01-03", "l_receiptdate": "1995-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thely regular dolphi" }
+{ "l_orderkey": 3555, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9235.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "are. slyly final foxes acro" }
+{ "l_orderkey": 3684, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-09-02", "l_receiptdate": "1993-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its boost alongside" }
+{ "l_orderkey": 3747, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 32835.84, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests shall h" }
+{ "l_orderkey": 3879, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46175.4, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-18", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly according to the expr" }
+{ "l_orderkey": 3907, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests according to the slyly pending " }
+{ "l_orderkey": 4004, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 20522.4, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-06-14", "l_receiptdate": "1993-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". ironic deposits cajole blithely?" }
+{ "l_orderkey": 4358, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48227.64, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "refully busy dep" }
+{ "l_orderkey": 4614, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6156.72, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ake quickly quickly regular epitap" }
+{ "l_orderkey": 5027, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32835.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-29", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cording to" }
+{ "l_orderkey": 5413, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1997-11-20", "l_receiptdate": "1998-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " theodolites. furiously ironic instr" }
+{ "l_orderkey": 5700, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23600.76, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-01-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly carefully fluffy hockey" }
+{ "l_orderkey": 37, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the final requests. ca" }
+{ "l_orderkey": 163, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12325.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly blithe accounts cajole " }
+{ "l_orderkey": 646, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t blithely regular deposits. quic" }
+{ "l_orderkey": 1316, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ges haggle of the" }
+{ "l_orderkey": 1671, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12325.44, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-17", "l_commitdate": "1996-09-02", "l_receiptdate": "1996-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "special, ironic" }
+{ "l_orderkey": 1827, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4108.48, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-10", "l_receiptdate": "1996-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. blithely" }
+{ "l_orderkey": 2023, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9244.08, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-06-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular pinto beans poa" }
+{ "l_orderkey": 2087, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "the quickly idle acco" }
+{ "l_orderkey": 2243, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10271.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-26", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "express, daring foxes affix fur" }
+{ "l_orderkey": 2434, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r deposits sleep furiou" }
+{ "l_orderkey": 2469, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8216.96, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-20", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. regular" }
+{ "l_orderkey": 2945, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28759.36, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le slyly along the eve" }
+{ "l_orderkey": 2978, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 43139.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial requests nag blithely alongside of th" }
+{ "l_orderkey": 3392, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34922.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-20", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e carefully even braids. " }
+{ "l_orderkey": 3394, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30813.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-12", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t ideas according to the fluffily iro" }
+{ "l_orderkey": 3588, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22596.64, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "inal accounts. pending, bo" }
+{ "l_orderkey": 3780, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25678.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-07-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l, unusual " }
+{ "l_orderkey": 3808, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48274.64, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fully for the quickly final deposits: flu" }
+{ "l_orderkey": 3876, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42111.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-10-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " quickly blit" }
+{ "l_orderkey": 4036, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20542.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-09-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly bold deposits cajole pending, blithe" }
+{ "l_orderkey": 4870, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6162.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-09", "l_commitdate": "1994-10-16", "l_receiptdate": "1994-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ress requests. bold, silent pinto bea" }
+{ "l_orderkey": 5281, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23623.76, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1996-01-26", "l_receiptdate": "1996-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". final theodolites cajole. ironic p" }
+{ "l_orderkey": 5606, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-11", "l_commitdate": "1997-01-13", "l_receiptdate": "1997-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ter the ironic accounts. even, ironic depos" }
+{ "l_orderkey": 5798, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2054.24, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e furiously across " }
+{ "l_orderkey": 5926, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts integrate. courts haggl" }
+{ "l_orderkey": 388, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47293.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-01-26", "l_receiptdate": "1993-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans nag about the careful reque" }
+{ "l_orderkey": 390, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13365.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sleep carefully idle packages. blithely " }
+{ "l_orderkey": 711, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47293.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1993-11-19", "l_receiptdate": "1994-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "deposits. permanen" }
+{ "l_orderkey": 711, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20562.4, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-17", "l_commitdate": "1993-11-10", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "kly regular acco" }
+{ "l_orderkey": 871, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1996-01-12", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar ideas-- slyly even accou" }
+{ "l_orderkey": 1028, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2056.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s alongside of the regular asymptotes sleep" }
+{ "l_orderkey": 1538, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43181.04, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-10", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests cajole blithely " }
+{ "l_orderkey": 1668, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-08", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "even platelets across the silent " }
+{ "l_orderkey": 1953, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ular, regular i" }
+{ "l_orderkey": 2503, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40096.68, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d carefully fluffily" }
+{ "l_orderkey": 2566, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1028.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "theodolites wake pending" }
+{ "l_orderkey": 2593, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6168.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-28", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ular packages. re" }
+{ "l_orderkey": 2629, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29815.48, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eposits serve unusual, express i" }
+{ "l_orderkey": 2662, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ajole carefully. sp" }
+{ "l_orderkey": 3168, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13365.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-05", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ironic somas haggle quick" }
+{ "l_orderkey": 3396, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34956.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-30", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": ". slyly unusual packages wak" }
+{ "l_orderkey": 3527, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17478.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular instruction" }
+{ "l_orderkey": 3650, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44209.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gside of the quick" }
+{ "l_orderkey": 4096, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20562.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-13", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sual requests. furiously bold packages wake" }
+{ "l_orderkey": 4257, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33927.96, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uffily regular accounts ar" }
+{ "l_orderkey": 4614, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42152.92, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ackages haggle carefully about the even, b" }
+{ "l_orderkey": 4677, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "unts doubt furiousl" }
+{ "l_orderkey": 4710, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48321.64, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely express packages. even, ironic re" }
+{ "l_orderkey": 4804, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45237.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "aggle quickly among the slyly fi" }
+{ "l_orderkey": 4996, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12337.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-22", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usly bold requests sleep dogge" }
+{ "l_orderkey": 5185, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-02", "l_receiptdate": "1997-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sts around the slyly perma" }
+{ "l_orderkey": 5511, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 50377.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-21", "l_commitdate": "1995-01-27", "l_receiptdate": "1994-12-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "bout the requests. theodolites " }
+{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
+{ "l_orderkey": 130, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14407.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-15", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-09-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. final instruction" }
+{ "l_orderkey": 229, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29844.48, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s, final request" }
+{ "l_orderkey": 290, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2058.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". permanently furious reques" }
+{ "l_orderkey": 929, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-06", "l_receiptdate": "1993-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ges haggle careful" }
+{ "l_orderkey": 995, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28815.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "pades. quick, final frays use flu" }
+{ "l_orderkey": 1219, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4116.48, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-24", "l_commitdate": "1995-11-22", "l_receiptdate": "1995-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lly quick requests. blithely even h" }
+{ "l_orderkey": 1280, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17495.04, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-04", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-02-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ructions integrate across the th" }
+{ "l_orderkey": 1318, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31902.72, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ve the carefully expr" }
+{ "l_orderkey": 2018, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23669.76, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ingly even theodolites s" }
+{ "l_orderkey": 2022, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20582.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-31", "l_commitdate": "1992-04-17", "l_receiptdate": "1992-04-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "r deposits kindle " }
+{ "l_orderkey": 2149, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-05-11", "l_receiptdate": "1993-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uriously final pac" }
+{ "l_orderkey": 2177, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-02-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". theodolites haggle carefu" }
+{ "l_orderkey": 2274, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-22", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " express packages. even accounts hagg" }
+{ "l_orderkey": 2723, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41164.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-11-22", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unwind fluffily carefully regular realms." }
+{ "l_orderkey": 3175, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12349.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-10-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ter the pending deposits. slyly e" }
+{ "l_orderkey": 3300, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3087.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-02", "l_receiptdate": "1995-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "g according to the dugouts. caref" }
+{ "l_orderkey": 3462, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6174.72, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-06-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "iously regular fo" }
+{ "l_orderkey": 3557, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38077.44, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-16", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gside of the ca" }
+{ "l_orderkey": 3749, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9262.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses cajole blithely pla" }
+{ "l_orderkey": 4069, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven theodolites nag quickly. fluffi" }
+{ "l_orderkey": 4197, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51456.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-11-01", "l_receiptdate": "1996-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully bold asymptotes nag blithe" }
+{ "l_orderkey": 4647, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34990.08, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly sly accounts" }
+{ "l_orderkey": 4993, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "pending, regular requests solve caref" }
+{ "l_orderkey": 5063, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31902.72, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-06-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages. ironic, ironic courts wake. carefu" }
+{ "l_orderkey": 5090, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47339.52, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lose theodolites sleep blit" }
+{ "l_orderkey": 5350, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-25", "l_commitdate": "1993-12-27", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es. blithe theodolites haggl" }
+{ "l_orderkey": 5411, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15436.8, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "attainments sleep slyly ironic" }
+{ "l_orderkey": 5543, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l excuses are furiously. slyly unusual requ" }
+{ "l_orderkey": 5637, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "oss the carefully express warhorses" }
+{ "l_orderkey": 5671, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47339.52, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lar pinto beans detect care" }
+{ "l_orderkey": 5671, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13378.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-03-26", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ers according to the ironic, unusual excu" }
+{ "l_orderkey": 5699, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "rmanent packages sleep across the f" }
+{ "l_orderkey": 5829, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6174.72, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-22", "l_commitdate": "1997-03-12", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sts. slyly special fo" }
+{ "l_orderkey": 5953, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37048.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " cajole furio" }
+{ "l_orderkey": 256, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46355.85, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " grouches. ideas wake quickly ar" }
+{ "l_orderkey": 451, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37084.68, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "rges can haggle carefully ironic, dogged " }
+{ "l_orderkey": 583, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express req" }
+{ "l_orderkey": 644, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11331.43, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ealthy pinto beans use carefu" }
+{ "l_orderkey": 864, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-23", "l_receiptdate": "1998-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gside of the furiously special" }
+{ "l_orderkey": 1447, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23692.99, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-25", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " dazzle quickly deposits. f" }
+{ "l_orderkey": 1538, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37084.68, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-11", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al deposits mo" }
+{ "l_orderkey": 1856, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 43265.46, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usly final deposits" }
+{ "l_orderkey": 2051, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49446.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-04", "l_commitdate": "1996-06-14", "l_receiptdate": "1996-05-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts. pending platelets believe about" }
+{ "l_orderkey": 2179, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22662.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lphins cajole acr" }
+{ "l_orderkey": 2434, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28843.64, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-28", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ven theodolites around the slyly" }
+{ "l_orderkey": 2499, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6180.78, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-12-14", "l_receiptdate": "1995-12-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cording to the" }
+{ "l_orderkey": 2886, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47385.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ously final packages sleep blithely regular" }
+{ "l_orderkey": 2950, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32964.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-21", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its wake carefully slyly final ideas." }
+{ "l_orderkey": 3009, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26783.38, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-06-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uriously specia" }
+{ "l_orderkey": 3459, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30903.9, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-22", "l_commitdate": "1994-09-12", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nic theodolites; evenly i" }
+{ "l_orderkey": 3460, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 48416.11, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es haggle slyly regular accounts. fi" }
+{ "l_orderkey": 3522, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7210.91, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e stealthil" }
+{ "l_orderkey": 3713, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14421.82, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eposits impress according" }
+{ "l_orderkey": 3782, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41205.2, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s instructions. regular accou" }
+{ "l_orderkey": 3815, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11331.43, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1997-11-05", "l_receiptdate": "1997-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sleep blithe" }
+{ "l_orderkey": 4386, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10301.3, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "gainst the quickly expre" }
+{ "l_orderkey": 4386, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40175.07, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions cajole quickly express" }
+{ "l_orderkey": 4705, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31934.03, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-03", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "furiously final accou" }
+{ "l_orderkey": 4774, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 30903.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "tions against the blithely final theodolit" }
+{ "l_orderkey": 4994, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 31934.03, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lar decoys cajole fluffil" }
+{ "l_orderkey": 5223, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 41205.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-01", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly pending " }
+{ "l_orderkey": 5255, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42235.33, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tect blithely against t" }
+{ "l_orderkey": 225, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3093.39, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-25", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " fluffily about the carefully bold a" }
+{ "l_orderkey": 386, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38151.81, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-03-01", "l_receiptdate": "1995-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ending pearls breach fluffily. slyly pen" }
+{ "l_orderkey": 515, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-10-08", "l_receiptdate": "1993-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r sauternes boost. final theodolites wake a" }
+{ "l_orderkey": 802, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45369.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rmanently idly special requ" }
+{ "l_orderkey": 994, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 25778.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-03", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual pinto beans." }
+{ "l_orderkey": 1061, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 51556.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nding excuses are around the e" }
+{ "l_orderkey": 1187, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15466.95, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1993-01-13", "l_receiptdate": "1993-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ests. foxes wake. carefu" }
+{ "l_orderkey": 1793, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uctions; depo" }
+{ "l_orderkey": 2116, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "r theodolites use blithely about the ir" }
+{ "l_orderkey": 2407, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40214.07, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-20", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "iously final deposits solv" }
+{ "l_orderkey": 2529, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4124.52, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al dependencies haggle slyly alongsi" }
+{ "l_orderkey": 2627, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggedly final excuses nag packages. f" }
+{ "l_orderkey": 2755, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5155.65, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-27", "l_commitdate": "1992-04-07", "l_receiptdate": "1992-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e the furi" }
+{ "l_orderkey": 3200, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37120.68, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "f the carefu" }
+{ "l_orderkey": 3264, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35058.42, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "rns haggle carefully. blit" }
+{ "l_orderkey": 3269, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16498.08, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-04-06", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole. silent deposits are f" }
+{ "l_orderkey": 3328, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25778.25, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e unusual, r" }
+{ "l_orderkey": 3521, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-29", "l_commitdate": "1992-12-20", "l_receiptdate": "1993-02-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully duri" }
+{ "l_orderkey": 3749, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7217.91, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-04-05", "l_receiptdate": "1995-04-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he slyly ironic packages" }
+{ "l_orderkey": 3814, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7217.91, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es sleep furiou" }
+{ "l_orderkey": 4193, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38151.81, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-25", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-05-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "er the quickly regular dependencies wake" }
+{ "l_orderkey": 4292, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-03", "l_commitdate": "1992-02-24", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "bove the silently regula" }
+{ "l_orderkey": 4544, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41245.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-15", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " detect slyly. evenly pending instru" }
+{ "l_orderkey": 4866, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17529.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-26", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess packages doubt. even somas wake f" }
+{ "l_orderkey": 4965, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1993-11-20", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. requests sublate quickly " }
+{ "l_orderkey": 5158, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r requests sleep q" }
+{ "l_orderkey": 5255, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-27", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ajole blithely fluf" }
+{ "l_orderkey": 5347, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 21653.73, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly slyly final requests. careful" }
+{ "l_orderkey": 5441, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-09-22", "l_receiptdate": "1994-10-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ording to the furio" }
+{ "l_orderkey": 5445, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12373.56, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-02", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly pending pinto beans was slyly al" }
+{ "l_orderkey": 5666, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24747.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-03-09", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "on the carefully pending asympto" }
+{ "l_orderkey": 5702, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45369.72, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-28", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-12-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ake according to th" }
+{ "l_orderkey": 5763, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-13", "l_receiptdate": "1998-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ding instruct" }
+{ "l_orderkey": 225, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12385.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-04", "l_commitdate": "1995-07-15", "l_receiptdate": "1995-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual requests. bus" }
+{ "l_orderkey": 359, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17546.21, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts according to the blithely" }
+{ "l_orderkey": 384, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14449.82, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckages are slyly after the slyly specia" }
+{ "l_orderkey": 417, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uriously bol" }
+{ "l_orderkey": 423, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-01", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ccounts. blithely regular pack" }
+{ "l_orderkey": 802, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19610.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "old, furious" }
+{ "l_orderkey": 967, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10321.3, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "old pinto beans alongside of the exp" }
+{ "l_orderkey": 1219, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pecial, ironic requ" }
+{ "l_orderkey": 1668, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8257.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arefully regular tithes! slyl" }
+{ "l_orderkey": 2182, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "en platele" }
+{ "l_orderkey": 2375, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9289.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly against the packages. bold pinto bean" }
+{ "l_orderkey": 2658, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 42317.33, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-07", "l_commitdate": "1995-11-04", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eposits. furiously final theodolite" }
+{ "l_orderkey": 2853, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20642.6, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-06-16", "l_receiptdate": "1994-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e slyly silent foxes. express deposits sno" }
+{ "l_orderkey": 3008, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8257.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1996-01-12", "l_receiptdate": "1995-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly ironic foxes. regular requests h" }
+{ "l_orderkey": 3078, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25803.25, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-05-01", "l_receiptdate": "1993-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "express dinos. carefully ironic" }
+{ "l_orderkey": 3111, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31996.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kages detect express attainments" }
+{ "l_orderkey": 3172, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13417.69, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-08-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits haggle along the" }
+{ "l_orderkey": 3397, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 28899.64, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "counts around the final reques" }
+{ "l_orderkey": 3493, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10321.3, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-27", "l_commitdate": "1993-10-07", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hall have to integ" }
+{ "l_orderkey": 3587, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49542.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "beans. blithely final depe" }
+{ "l_orderkey": 3814, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 12385.56, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ages cajole. packages haggle. final" }
+{ "l_orderkey": 4419, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts wake slyly final dugou" }
+{ "l_orderkey": 4803, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-05-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular reque" }
+{ "l_orderkey": 5190, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "furiously regular pinto beans. furiously i" }
+{ "l_orderkey": 5381, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 34060.29, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-09", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly special deposits " }
+{ "l_orderkey": 5507, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 49542.24, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uriously regular acc" }
+{ "l_orderkey": 5607, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23738.99, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the special, final patterns " }
+{ "l_orderkey": 5696, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 38188.81, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully expres" }
+{ "l_orderkey": 5957, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29931.77, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sits. final, even asymptotes cajole quickly" }
+{ "l_orderkey": 5958, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33028.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully special theodolites. carefully " }
+{ "l_orderkey": 199, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51656.5, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "essly regular ideas boost sly" }
+{ "l_orderkey": 258, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly blithely special mul" }
+{ "l_orderkey": 517, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11364.43, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-01", "l_receiptdate": "1997-06-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly throughout the fu" }
+{ "l_orderkey": 802, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35126.42, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-03-15", "l_receiptdate": "1995-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "instructions cajole carefully. quietl" }
+{ "l_orderkey": 1125, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly express packages a" }
+{ "l_orderkey": 1284, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40292.07, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "even accoun" }
+{ "l_orderkey": 1472, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26861.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ic packages w" }
+{ "l_orderkey": 1506, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47523.98, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sits whithout the blithely ironic packages" }
+{ "l_orderkey": 2467, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7231.91, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-28", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular packages cajole " }
+{ "l_orderkey": 2499, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-10-28", "l_receiptdate": "1996-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "to beans across the carefully ironic theodo" }
+{ "l_orderkey": 2789, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 43391.46, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ending packages shoul" }
+{ "l_orderkey": 2791, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45457.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "heodolites use furio" }
+{ "l_orderkey": 2980, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27894.51, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " theodolites cajole blithely sl" }
+{ "l_orderkey": 3040, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9298.17, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ges. pending packages wake. requests" }
+{ "l_orderkey": 3140, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28927.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lar ideas. slyly ironic d" }
+{ "l_orderkey": 3458, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49590.24, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-17", "l_commitdate": "1995-01-25", "l_receiptdate": "1995-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously pending dep" }
+{ "l_orderkey": 3523, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-18", "l_receiptdate": "1998-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts. final accounts detect furiously along " }
+{ "l_orderkey": 3809, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33060.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "xcuses would boost against the fluffily eve" }
+{ "l_orderkey": 4453, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16530.08, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ar excuses nag quickly even accounts. b" }
+{ "l_orderkey": 4512, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44424.59, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1995-11-28", "l_receiptdate": "1996-01-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "are carefully. theodolites wake" }
+{ "l_orderkey": 4544, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-09-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dolites detect quickly reg" }
+{ "l_orderkey": 4546, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10331.3, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-10-10", "l_receiptdate": "1995-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits alongside of the" }
+{ "l_orderkey": 4711, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7231.91, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-05-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly. bold accounts use fluff" }
+{ "l_orderkey": 4864, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46490.85, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "round the furiously careful pa" }
+{ "l_orderkey": 4964, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 29960.77, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-08-30", "l_receiptdate": "1997-11-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "k accounts nag carefully-- ironic, fin" }
+{ "l_orderkey": 5120, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28927.64, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-08-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " across the silent requests. caref" }
+{ "l_orderkey": 5414, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15496.95, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-18", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e slyly about the carefully regula" }
+{ "l_orderkey": 5762, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48557.11, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-03-23", "l_receiptdate": "1997-03-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "equests sleep after the furiously ironic pa" }
+{ "l_orderkey": 134, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12409.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly regular pac" }
+{ "l_orderkey": 199, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31023.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-04-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ilent packages doze quickly. thinly " }
+{ "l_orderkey": 421, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1034.13, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oldly busy deposit" }
+{ "l_orderkey": 518, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12409.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-04-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages thrash slyly" }
+{ "l_orderkey": 644, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47569.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-20", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " special requests was sometimes expre" }
+{ "l_orderkey": 773, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9307.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ent orbits haggle fluffily after the " }
+{ "l_orderkey": 838, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously final ideas. slow, bold " }
+{ "l_orderkey": 1317, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35160.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-13", "l_commitdate": "1995-08-08", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "deposits boost thinly blithely final id" }
+{ "l_orderkey": 1408, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43433.46, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even packages. even accounts cajole" }
+{ "l_orderkey": 1666, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32058.03, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-11", "l_receiptdate": "1996-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ding to the express, bold accounts. fu" }
+{ "l_orderkey": 1892, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nts. slyly regular asymptot" }
+{ "l_orderkey": 1985, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-11-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "regular requests. furiously express" }
+{ "l_orderkey": 2023, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "its! carefully ex" }
+{ "l_orderkey": 2054, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17580.21, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-06-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ges nag acc" }
+{ "l_orderkey": 2055, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16546.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully daringly regular accounts." }
+{ "l_orderkey": 2433, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lithely blithely final ide" }
+{ "l_orderkey": 2853, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26887.38, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-06-05", "l_receiptdate": "1994-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "dolphins wake slyly. blith" }
+{ "l_orderkey": 2882, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33092.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. quickly regular e" }
+{ "l_orderkey": 2976, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13443.69, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-06", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-02-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " furiously final courts boost " }
+{ "l_orderkey": 3207, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33092.16, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l deposits wake beyond the carefully" }
+{ "l_orderkey": 3297, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10341.3, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1993-01-21", "l_receiptdate": "1992-12-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ironic idea" }
+{ "l_orderkey": 3332, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27921.51, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ording to the slyly regula" }
+{ "l_orderkey": 3399, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28955.64, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-29", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "oggedly final theodolites grow. fi" }
+{ "l_orderkey": 3523, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 49638.24, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " regular requests" }
+{ "l_orderkey": 3685, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39296.94, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "thely unusual pack" }
+{ "l_orderkey": 3750, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-28", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "usly busy account" }
+{ "l_orderkey": 4039, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44467.59, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-20", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts along the regular in" }
+{ "l_orderkey": 5152, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-10", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the final deposits. slyly ironic warth" }
+{ "l_orderkey": 5477, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32058.03, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special Tiresias cajole furiously. pending" }
+{ "l_orderkey": 5636, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 24819.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "counts sleep furiously b" }
+{ "l_orderkey": 5798, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 22750.86, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sits poach carefully" }
+{ "l_orderkey": 5893, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 44467.59, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-09-27", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. regular courts above the carefully silen" }
+{ "l_orderkey": 99, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43475.46, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-19", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages are fluffily furiously ir" }
+{ "l_orderkey": 353, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12421.56, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g deposits cajole " }
+{ "l_orderkey": 935, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37264.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "leep about the exp" }
+{ "l_orderkey": 1095, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28983.64, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " regular pac" }
+{ "l_orderkey": 1254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckages boost. furious warhorses cajole" }
+{ "l_orderkey": 1445, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24843.12, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-05-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rate after the carefully reg" }
+{ "l_orderkey": 1508, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30018.77, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r instructions. carefully" }
+{ "l_orderkey": 1699, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17597.21, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "haggle blithely slyly" }
+{ "l_orderkey": 1826, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ously? quickly pe" }
+{ "l_orderkey": 1925, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-11", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts. carefully ironic packages boost ab" }
+{ "l_orderkey": 2052, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ts according t" }
+{ "l_orderkey": 2055, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12421.56, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al pains. acco" }
+{ "l_orderkey": 2211, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3105.39, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-28", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pendencies after the regular f" }
+{ "l_orderkey": 2276, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13456.69, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-24", "l_commitdate": "1996-06-18", "l_receiptdate": "1996-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully ironic foxes cajole q" }
+{ "l_orderkey": 2532, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1035.13, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-11-23", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely final ideas cajole despite the ca" }
+{ "l_orderkey": 2567, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 44510.59, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "requests. final courts cajole " }
+{ "l_orderkey": 2659, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24843.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle carefully " }
+{ "l_orderkey": 3172, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28983.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-07-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "regular ideas. packages are furi" }
+{ "l_orderkey": 3232, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35194.42, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-14", "l_receiptdate": "1993-02-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old packages integrate quickly " }
+{ "l_orderkey": 3684, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13456.69, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-23", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-08-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing, unusual pinto beans! thinly p" }
+{ "l_orderkey": 3844, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es haggle final acco" }
+{ "l_orderkey": 4192, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7245.91, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y; excuses use. ironic, close instru" }
+{ "l_orderkey": 4422, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5175.65, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e furiously about t" }
+{ "l_orderkey": 4486, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. specia" }
+{ "l_orderkey": 5063, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18632.34, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "refully quiet reques" }
+{ "l_orderkey": 5219, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " blithely according to the stea" }
+{ "l_orderkey": 5254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10351.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-19", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " accounts. silent deposit" }
+{ "l_orderkey": 5572, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-08", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully regular platelet" }
+{ "l_orderkey": 5959, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 50721.37, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usual packages haggle slyly pi" }
+{ "l_orderkey": 96, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-03", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e quickly even ideas. furiou" }
+{ "l_orderkey": 129, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35228.42, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-29", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. express ideas" }
+{ "l_orderkey": 196, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19686.47, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-04-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sts maintain foxes. furiously regular p" }
+{ "l_orderkey": 486, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26939.38, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-05-25", "l_receiptdate": "1996-03-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "forges along the " }
+{ "l_orderkey": 866, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-01-14", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tegrate fluffily. carefully f" }
+{ "l_orderkey": 1027, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22794.86, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "the furiously express ex" }
+{ "l_orderkey": 1061, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42481.33, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-29", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s are. ironic theodolites cajole. dep" }
+{ "l_orderkey": 1095, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24867.12, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "packages nod furiously above the carefully " }
+{ "l_orderkey": 1153, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 26939.38, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-16", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-09-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kages haggle carefully. f" }
+{ "l_orderkey": 1312, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29011.64, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-09", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously final frays should use quick" }
+{ "l_orderkey": 1574, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14505.82, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-19", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily bold a" }
+{ "l_orderkey": 1733, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39372.94, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-08-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " deposits " }
+{ "l_orderkey": 1831, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-01-27", "l_receiptdate": "1993-12-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "mptotes. furiously regular dolphins al" }
+{ "l_orderkey": 2084, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38336.81, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y careful courts." }
+{ "l_orderkey": 2373, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3108.39, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dependencies wake ironical" }
+{ "l_orderkey": 2786, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-19", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "low deposits are ironic" }
+{ "l_orderkey": 2915, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al requests haggle furiousl" }
+{ "l_orderkey": 2951, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24867.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-04-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " ironic multipliers. express, regular" }
+{ "l_orderkey": 3106, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lets. quietly regular courts " }
+{ "l_orderkey": 3332, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21758.73, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " quick packages sle" }
+{ "l_orderkey": 3366, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ackages sleep carefully across the bli" }
+{ "l_orderkey": 3428, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 48698.11, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-05-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y final pinto " }
+{ "l_orderkey": 3650, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-26", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckly special platelets. furiously sil" }
+{ "l_orderkey": 3782, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1996-10-31", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "slyly even pinto beans hag" }
+{ "l_orderkey": 4035, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4144.52, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en instructions sleep blith" }
+{ "l_orderkey": 4384, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "instructions sleep. blithely express pa" }
+{ "l_orderkey": 4707, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " alongside of the slyly ironic instructio" }
+{ "l_orderkey": 4805, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18650.34, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "o use pending, unusu" }
+{ "l_orderkey": 4871, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 37300.68, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-29", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ely according" }
+{ "l_orderkey": 5153, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43517.46, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-19", "l_commitdate": "1995-11-23", "l_receiptdate": "1995-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ickly even deposi" }
+{ "l_orderkey": 5316, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32120.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. deposits cajole around t" }
+{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45589.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "efully ironi" }
+{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-05-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ove the regula" }
+{ "l_orderkey": 5763, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23830.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "re after the blithel" }
+{ "l_orderkey": 5986, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6216.78, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al foxes within the slyly speci" }
+{ "l_orderkey": 135, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20742.6, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-02-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "theodolites. quickly p" }
+{ "l_orderkey": 387, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1037.13, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-05-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " pinto beans wake furiously carefu" }
+{ "l_orderkey": 485, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 22816.86, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "refully final notornis haggle according " }
+{ "l_orderkey": 547, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "thely specia" }
+{ "l_orderkey": 614, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-01-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " regular platelets cajole quickly eve" }
+{ "l_orderkey": 675, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-19", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. furiously expre" }
+{ "l_orderkey": 993, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-11-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es. ironic, ironic requests" }
+{ "l_orderkey": 1029, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46670.85, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sits boost blithely" }
+{ "l_orderkey": 1062, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-03-07", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. pending acc" }
+{ "l_orderkey": 1095, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34225.29, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-09-22", "l_receiptdate": "1995-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly around the iron" }
+{ "l_orderkey": 1573, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7259.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-30", "l_commitdate": "1993-03-14", "l_receiptdate": "1993-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eodolites sleep slyly. slyly f" }
+{ "l_orderkey": 1703, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he carefully" }
+{ "l_orderkey": 1760, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "instructions poach slyly ironic theodolites" }
+{ "l_orderkey": 1795, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-28", "l_commitdate": "1994-05-24", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ites sleep carefully slyly p" }
+{ "l_orderkey": 1955, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g to the carefully sile" }
+{ "l_orderkey": 2277, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully bold" }
+{ "l_orderkey": 2309, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9334.17, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-10", "l_receiptdate": "1996-01-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ding, unusual instructions. dep" }
+{ "l_orderkey": 2656, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions wake along the furio" }
+{ "l_orderkey": 2661, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42522.33, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-03-27", "l_receiptdate": "1997-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "iously ironically ironic requests. " }
+{ "l_orderkey": 2753, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37336.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle slyly final c" }
+{ "l_orderkey": 3111, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22816.86, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. regular dolphins against the " }
+{ "l_orderkey": 3429, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " haggle furiously ir" }
+{ "l_orderkey": 3524, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5185.65, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts whithout the bold depende" }
+{ "l_orderkey": 3617, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11408.43, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly quickly even requests. final" }
+{ "l_orderkey": 3652, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38373.81, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "osits haggle carefu" }
+{ "l_orderkey": 3745, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18668.34, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-16", "l_receiptdate": "1993-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slyly bold pinto beans according to " }
+{ "l_orderkey": 3810, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42522.33, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-26", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l requests boost slyly along the slyl" }
+{ "l_orderkey": 3936, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25928.25, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-03", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gular requests nag quic" }
+{ "l_orderkey": 3940, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11408.43, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e of the special packages. furiously" }
+{ "l_orderkey": 4099, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3111.39, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". special packages sleep" }
+{ "l_orderkey": 4102, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7259.91, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "bove the carefully pending the" }
+{ "l_orderkey": 4290, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23853.99, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uests cajole carefully." }
+{ "l_orderkey": 4484, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40448.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-04-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic accounts wake blithel" }
+{ "l_orderkey": 4612, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 41485.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special platelets." }
+{ "l_orderkey": 4865, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4148.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-24", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sts. blithely special instruction" }
+{ "l_orderkey": 5635, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly even" }
+{ "l_orderkey": 5696, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29039.64, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-03", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the fluffily brave pearls " }
+{ "l_orderkey": 5794, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48745.11, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "quests. blithely final excu" }
+{ "l_orderkey": 33, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5190.65, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". stealthily bold exc" }
+{ "l_orderkey": 69, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17648.21, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-02", "l_commitdate": "1994-07-07", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "final, pending instr" }
+{ "l_orderkey": 226, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47753.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. carefully bold accounts cajol" }
+{ "l_orderkey": 291, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19724.47, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-04-25", "l_receiptdate": "1994-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e. ruthlessly final accounts after the" }
+{ "l_orderkey": 448, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-10-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ious, final gifts" }
+{ "l_orderkey": 482, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33220.16, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "usual deposits affix against " }
+{ "l_orderkey": 576, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5190.65, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l foxes boost slyly. accounts af" }
+{ "l_orderkey": 1125, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1994-12-02", "l_receiptdate": "1995-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es about the slyly s" }
+{ "l_orderkey": 1281, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "dencies. thinly final pinto beans wake" }
+{ "l_orderkey": 1283, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests use along the fluff" }
+{ "l_orderkey": 1889, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l pinto beans kindle " }
+{ "l_orderkey": 1991, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6228.78, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uickly blithely final de" }
+{ "l_orderkey": 2272, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31143.9, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-08-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "quests at the foxes haggle evenly pack" }
+{ "l_orderkey": 2309, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 49830.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts. id" }
+{ "l_orderkey": 2340, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-01", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": ". carefully ironic" }
+{ "l_orderkey": 2848, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8305.04, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-07-09", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly regular foxes. " }
+{ "l_orderkey": 2951, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ep about the final, even package" }
+{ "l_orderkey": 3010, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ounts. pendin" }
+{ "l_orderkey": 3235, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffy pinto bea" }
+{ "l_orderkey": 3329, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-06", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts at the re" }
+{ "l_orderkey": 3362, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-08-28", "l_receiptdate": "1995-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "es against the quickly permanent pint" }
+{ "l_orderkey": 3396, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly special foxes. accounts wake careful" }
+{ "l_orderkey": 3525, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28029.51, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y slyly special asymptotes" }
+{ "l_orderkey": 3616, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. furiously ev" }
+{ "l_orderkey": 4007, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15571.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "le furiously quickly " }
+{ "l_orderkey": 4065, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14533.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-22", "l_commitdate": "1994-07-29", "l_receiptdate": "1994-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e furiously outside " }
+{ "l_orderkey": 4132, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pths wake against the stealthily special pi" }
+{ "l_orderkey": 4161, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43601.46, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-12", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thely across the even attainments. express" }
+{ "l_orderkey": 4487, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38410.81, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "bove the fu" }
+{ "l_orderkey": 4513, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35296.42, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sits. quickly even instructions " }
+{ "l_orderkey": 4832, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44639.59, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-02-20", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oze according to the accou" }
+{ "l_orderkey": 4934, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aggle furiously among the busily final re" }
+{ "l_orderkey": 4965, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "iously slyly" }
+{ "l_orderkey": 5157, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-06", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-09-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits nag blithely. final reque" }
+{ "l_orderkey": 5189, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45677.72, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-02-07", "l_receiptdate": "1994-01-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y finally pendin" }
+{ "l_orderkey": 5248, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 46715.85, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". bold, pending foxes h" }
+{ "l_orderkey": 5286, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-12-21", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. express foxes of the" }
+{ "l_orderkey": 5479, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51906.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-24", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic gifts. even dependencies sno" }
+{ "l_orderkey": 5507, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49830.24, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly idle deposits. final, final fox" }
+{ "l_orderkey": 5573, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44639.59, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " bold package" }
+{ "l_orderkey": 5638, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46715.85, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-17", "l_commitdate": "1994-03-09", "l_receiptdate": "1994-06-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar foxes. fluffily pending accounts " }
+{ "l_orderkey": 5664, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-29", "l_commitdate": "1998-09-17", "l_receiptdate": "1998-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "d the final " }
+{ "l_orderkey": 101, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12469.56, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-29", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-04-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". quickly regular" }
+{ "l_orderkey": 544, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48839.11, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial pains. deposits grow foxes. " }
+{ "l_orderkey": 710, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "xpress, special ideas. bl" }
+{ "l_orderkey": 867, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1993-12-25", "l_receiptdate": "1994-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "pendencies-- slyly unusual packages hagg" }
+{ "l_orderkey": 1059, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38447.81, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " packages lose in place of the slyly unusu" }
+{ "l_orderkey": 1696, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13508.69, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-01", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-03-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tructions play slyly q" }
+{ "l_orderkey": 1731, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-11", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fily quick asymptotes" }
+{ "l_orderkey": 1735, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50917.37, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1993-02-03", "l_receiptdate": "1993-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y express accounts above the exp" }
+{ "l_orderkey": 1956, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-11-26", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r theodolites sleep above the b" }
+{ "l_orderkey": 2177, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28056.51, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "even, regula" }
+{ "l_orderkey": 2179, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20782.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ncies. fin" }
+{ "l_orderkey": 2309, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47799.98, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly according to the carefully " }
+{ "l_orderkey": 2534, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30134.77, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-09", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ugouts haggle slyly. final" }
+{ "l_orderkey": 2596, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44682.59, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ial packages haggl" }
+{ "l_orderkey": 2823, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 49878.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously busily slow excus" }
+{ "l_orderkey": 2853, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14547.82, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oach slyly along t" }
+{ "l_orderkey": 2880, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27017.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-15", "l_receiptdate": "1992-04-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ully among the regular warthogs" }
+{ "l_orderkey": 3013, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31173.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ronic packages. slyly even" }
+{ "l_orderkey": 3074, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-08", "l_commitdate": "1993-01-28", "l_receiptdate": "1992-12-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously throu" }
+{ "l_orderkey": 3171, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51956.5, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously final foxes about the ca" }
+{ "l_orderkey": 3328, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45721.72, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-12-19", "l_receiptdate": "1992-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "dly quickly final foxes? re" }
+{ "l_orderkey": 3427, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 41565.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-08-19", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "patterns cajole ca" }
+{ "l_orderkey": 3747, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31173.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-15", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "! furiously f" }
+{ "l_orderkey": 3910, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10391.3, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-10-31", "l_receiptdate": "1996-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tions boost furiously unusual e" }
+{ "l_orderkey": 4066, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9352.17, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "nal, ironic accounts. blithel" }
+{ "l_orderkey": 4099, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "onic foxes. quickly final fox" }
+{ "l_orderkey": 4354, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18704.34, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-12-11", "l_receiptdate": "1994-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ross the furiously " }
+{ "l_orderkey": 4578, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "odolites. carefully unusual ideas accor" }
+{ "l_orderkey": 4647, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2078.26, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-07-22", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dolites wake furiously special pinto be" }
+{ "l_orderkey": 4992, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23899.99, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-28", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly regul" }
+{ "l_orderkey": 5251, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37408.68, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "slowly! bli" }
+{ "l_orderkey": 5252, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gular requests." }
+{ "l_orderkey": 5732, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27017.38, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "totes cajole according to the theodolites." }
+{ "l_orderkey": 5765, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32213.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the furiou" }
+{ "l_orderkey": 6, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "p furiously special foxes" }
+{ "l_orderkey": 68, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42645.74, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-07-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "eposits nag special ideas. furiousl" }
+{ "l_orderkey": 165, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 50966.86, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-20", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "uses sleep slyly ruthlessly regular a" }
+{ "l_orderkey": 1058, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24963.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully ironic accounts. express accou" }
+{ "l_orderkey": 1155, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12481.68, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1998-01-03", "l_receiptdate": "1997-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages do" }
+{ "l_orderkey": 1218, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16642.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ven realms be" }
+{ "l_orderkey": 1537, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3120.42, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-20", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s, final ideas detect sl" }
+{ "l_orderkey": 1700, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39525.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ular dependencies engage slyly " }
+{ "l_orderkey": 1921, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21842.94, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-03-28", "l_receiptdate": "1994-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ckly regula" }
+{ "l_orderkey": 2116, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic dependencies around the iro" }
+{ "l_orderkey": 2211, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41605.6, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-10-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "posits among the express dolphins" }
+{ "l_orderkey": 2307, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously. furiously furious requ" }
+{ "l_orderkey": 2690, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of th" }
+{ "l_orderkey": 2757, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26003.5, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-11-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uickly regular " }
+{ "l_orderkey": 2881, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7280.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ironic packages are carefully final ac" }
+{ "l_orderkey": 3110, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "side of the blithely unusual courts. slyly " }
+{ "l_orderkey": 3296, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-12-20", "l_receiptdate": "1994-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular deposits. quic" }
+{ "l_orderkey": 3335, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16642.24, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "g packages. carefully regular reque" }
+{ "l_orderkey": 3618, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39525.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-22", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nts haggle fluffily above the regular " }
+{ "l_orderkey": 3623, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13521.82, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-02-26", "l_receiptdate": "1997-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "deas. furiously expres" }
+{ "l_orderkey": 3872, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 41605.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1996-10-29", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nts? regularly ironic ex" }
+{ "l_orderkey": 3873, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30164.06, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-22", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "olphins af" }
+{ "l_orderkey": 3876, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-30", "l_commitdate": "1996-10-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t dependencies. blithely final packages u" }
+{ "l_orderkey": 4102, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-06", "l_receiptdate": "1996-06-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y among the furiously special" }
+{ "l_orderkey": 4320, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6240.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-01-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "against the carefully careful asym" }
+{ "l_orderkey": 4322, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9361.26, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ual instructio" }
+{ "l_orderkey": 4386, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4160.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully iron" }
+{ "l_orderkey": 4871, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 10401.4, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "p ironic theodolites. slyly even platel" }
+{ "l_orderkey": 4896, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-12-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e after the slowly f" }
+{ "l_orderkey": 4934, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8321.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-04-30", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "arefully express pains cajo" }
+{ "l_orderkey": 5092, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13521.82, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1996-01-05", "l_receiptdate": "1995-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es detect sly" }
+{ "l_orderkey": 5506, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-01-13", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "onic theodolites are fluffil" }
+{ "l_orderkey": 5669, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31204.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-07-28", "l_receiptdate": "1996-08-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l accounts. care" }
+{ "l_orderkey": 5698, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. even, ironic " }
+{ "l_orderkey": 5955, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14561.96, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " unusual, bold theodolit" }
+{ "l_orderkey": 132, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. platelets wake furio" }
+{ "l_orderkey": 321, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 42686.74, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "special packages shall have to doze blit" }
+{ "l_orderkey": 582, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43727.88, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-12-21", "l_receiptdate": "1997-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "iously beside the silent de" }
+{ "l_orderkey": 738, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-06-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ove the slyly regular p" }
+{ "l_orderkey": 836, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47892.44, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-04-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "boldly final pinto beans haggle furiously" }
+{ "l_orderkey": 1089, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1041.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-07-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "n courts among the caref" }
+{ "l_orderkey": 1344, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15617.1, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rding to the blithely ironic theodolite" }
+{ "l_orderkey": 1380, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41645.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly final frets. ironic," }
+{ "l_orderkey": 1604, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38522.18, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-09-21", "l_receiptdate": "1993-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "requests. blithely ironic somas s" }
+{ "l_orderkey": 1664, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 41645.6, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-02", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se blithely unusual pains. carefully" }
+{ "l_orderkey": 1730, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44769.02, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-26", "l_commitdate": "1998-10-22", "l_receiptdate": "1998-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ng deposits cajo" }
+{ "l_orderkey": 1890, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27069.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ngage. slyly ironic " }
+{ "l_orderkey": 2086, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-15", "l_commitdate": "1995-01-05", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e carefully along th" }
+{ "l_orderkey": 2311, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " fluffily even patterns haggle blithely. re" }
+{ "l_orderkey": 2373, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30193.06, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-14", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent ideas affix furiousl" }
+{ "l_orderkey": 2496, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39563.32, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " bold accounts. furi" }
+{ "l_orderkey": 2565, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 49974.72, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-05-06", "l_receiptdate": "1998-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "r instructions sleep qui" }
+{ "l_orderkey": 2820, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39563.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests despite the carefully unusual a" }
+{ "l_orderkey": 3264, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44769.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep at the blithely bold" }
+{ "l_orderkey": 3367, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35398.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-02-23", "l_receiptdate": "1993-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts wake slyly " }
+{ "l_orderkey": 3712, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28110.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ctions. even accounts haggle alongside " }
+{ "l_orderkey": 3747, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43727.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y. blithely fina" }
+{ "l_orderkey": 3751, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "rthogs could have to slee" }
+{ "l_orderkey": 3776, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 51015.86, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-16", "l_receiptdate": "1992-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "equests. final, thin grouches " }
+{ "l_orderkey": 3876, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y above the pending tithes. blithely ironi" }
+{ "l_orderkey": 4067, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1992-11-23", "l_receiptdate": "1993-01-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle slyly unusual, final" }
+{ "l_orderkey": 4166, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8329.12, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "uickly. blithely pending de" }
+{ "l_orderkey": 4228, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "f the slyly fluffy pinto beans are" }
+{ "l_orderkey": 4448, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3123.42, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ronic theod" }
+{ "l_orderkey": 4449, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10411.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-09", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-05-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts alongside of the platelets integr" }
+{ "l_orderkey": 4485, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47892.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1994-12-14", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". ironic foxes haggle. regular war" }
+{ "l_orderkey": 4512, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "counts are against the quickly regular " }
+{ "l_orderkey": 4583, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "romise. reques" }
+{ "l_orderkey": 4672, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-25", "l_receiptdate": "1995-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s boost at the ca" }
+{ "l_orderkey": 4901, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38522.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-26", "l_commitdate": "1998-02-20", "l_receiptdate": "1998-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " furiously ev" }
+{ "l_orderkey": 5252, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13534.82, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "boost fluffily across " }
+{ "l_orderkey": 5409, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-13", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "cross the sil" }
+{ "l_orderkey": 5536, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11452.54, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-18", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " snooze furio" }
+{ "l_orderkey": 192, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "equests. ideas sleep idea" }
+{ "l_orderkey": 225, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 45854.16, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-22", "l_commitdate": "1995-08-16", "l_receiptdate": "1995-10-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "leep slyly " }
+{ "l_orderkey": 390, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 43769.88, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "counts nag across the sly, sil" }
+{ "l_orderkey": 583, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34390.62, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-04-25", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "kages cajole slyly across the" }
+{ "l_orderkey": 998, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31264.2, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-02", "l_commitdate": "1995-01-23", "l_receiptdate": "1994-12-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lyly idle Tir" }
+{ "l_orderkey": 1153, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "oss the ex" }
+{ "l_orderkey": 1605, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-29", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". carefully r" }
+{ "l_orderkey": 1632, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. blithe, bold ideas cajo" }
+{ "l_orderkey": 1952, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-05-21", "l_receiptdate": "1994-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "packages haggle. " }
+{ "l_orderkey": 2081, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-19", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " silent, spe" }
+{ "l_orderkey": 2241, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9379.26, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lyly final " }
+{ "l_orderkey": 2306, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 43769.88, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "furiously final acco" }
+{ "l_orderkey": 2307, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25011.36, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "stealthily special packages nag a" }
+{ "l_orderkey": 2816, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4168.56, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-07", "l_receiptdate": "1995-01-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely pending id" }
+{ "l_orderkey": 3107, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36474.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-19", "l_receiptdate": "1997-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ets doubt furiously final ideas. final" }
+{ "l_orderkey": 3136, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31264.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "leep blithel" }
+{ "l_orderkey": 3395, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21884.94, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-01-13", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " careful dep" }
+{ "l_orderkey": 3458, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-02-02", "l_receiptdate": "1995-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites; regular theodolites cajole " }
+{ "l_orderkey": 3556, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-14", "l_commitdate": "1992-12-21", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ckages boost quickl" }
+{ "l_orderkey": 3719, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-15", "l_receiptdate": "1997-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the furiously special pinto bean" }
+{ "l_orderkey": 4036, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "equests wake about the bold id" }
+{ "l_orderkey": 4231, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "hely along the silent at" }
+{ "l_orderkey": 5158, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42727.74, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-19", "l_receiptdate": "1997-03-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "deposits. quickly special " }
+{ "l_orderkey": 5318, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "requests must sleep slyly quickly" }
+{ "l_orderkey": 5413, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38559.18, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-01", "l_receiptdate": "1997-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly bold instructions affix idly unusual, " }
+{ "l_orderkey": 5670, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11463.54, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "etect furiously among the even pin" }
+{ "l_orderkey": 263, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52157.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "re the packages. special" }
+{ "l_orderkey": 323, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9388.26, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nic accounts. regular, regular pack" }
+{ "l_orderkey": 450, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 33380.48, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-02", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " accounts nod fluffily even, pending" }
+{ "l_orderkey": 672, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 36509.9, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-07-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " dependencies haggle quickly. theo" }
+{ "l_orderkey": 676, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11474.54, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-09", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "he final acco" }
+{ "l_orderkey": 678, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16690.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "equests cajole around the carefully regular" }
+{ "l_orderkey": 708, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37553.04, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-04", "l_receiptdate": "1998-08-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ests. even, regular hockey p" }
+{ "l_orderkey": 802, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41725.6, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y bold accou" }
+{ "l_orderkey": 807, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31294.2, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "cial accoun" }
+{ "l_orderkey": 871, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27121.64, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-01-05", "l_receiptdate": "1996-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "symptotes use quickly near the " }
+{ "l_orderkey": 1154, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32337.34, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithely. final, blithe " }
+{ "l_orderkey": 1285, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 46941.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-05", "l_commitdate": "1992-08-08", "l_receiptdate": "1992-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special requests haggle blithely." }
+{ "l_orderkey": 1347, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35466.76, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-25", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "r packages. f" }
+{ "l_orderkey": 1542, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21905.94, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-13", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending foxes nag blithely " }
+{ "l_orderkey": 1638, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26078.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gle final, ironic pinto beans. " }
+{ "l_orderkey": 1763, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3129.42, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ut the slyly pending deposi" }
+{ "l_orderkey": 2176, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2086.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s pinto beans" }
+{ "l_orderkey": 2180, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23992.22, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ggle alongside of the fluffily speci" }
+{ "l_orderkey": 2307, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7301.98, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-09-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages cajo" }
+{ "l_orderkey": 2498, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50070.72, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-25", "l_commitdate": "1994-01-09", "l_receiptdate": "1993-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "onic requests wake" }
+{ "l_orderkey": 2913, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle. even, bold instructi" }
+{ "l_orderkey": 3109, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52157.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-16", "l_receiptdate": "1993-10-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular packages boost blithely even, re" }
+{ "l_orderkey": 3458, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37553.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s lose. blithely ironic requests boost" }
+{ "l_orderkey": 3524, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17733.38, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "g, final epitaphs about the pinto " }
+{ "l_orderkey": 3553, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4172.56, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-13", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "olites boost bli" }
+{ "l_orderkey": 4198, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47984.44, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits among th" }
+{ "l_orderkey": 4834, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39639.32, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "alongside of the carefully even plate" }
+{ "l_orderkey": 4897, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35466.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts. special dependencies use fluffily " }
+{ "l_orderkey": 4964, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18776.52, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " platelets. furio" }
+{ "l_orderkey": 5027, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3129.42, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-30", "l_commitdate": "1997-11-26", "l_receiptdate": "1997-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t the even mu" }
+{ "l_orderkey": 5094, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19819.66, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-06-12", "l_receiptdate": "1993-04-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ronic foxes. furi" }
+{ "l_orderkey": 5348, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14603.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "en pinto beans. somas cajo" }
+{ "l_orderkey": 5511, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23992.22, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-03", "l_commitdate": "1995-01-05", "l_receiptdate": "1995-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ully deposits. warthogs hagg" }
+{ "l_orderkey": 5543, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14603.96, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-09", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-10-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial reque" }
+{ "l_orderkey": 5729, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1994-11-11", "l_receiptdate": "1994-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. even sheaves nag courts. " }
+{ "l_orderkey": 5921, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 42768.74, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual, regular theodol" }
+{ "l_orderkey": 327, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-05", "l_commitdate": "1995-06-07", "l_receiptdate": "1995-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "cial ideas sleep af" }
+{ "l_orderkey": 1061, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 36544.9, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-05", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ending requests nag careful" }
+{ "l_orderkey": 1381, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49074.58, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-22", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic deposits" }
+{ "l_orderkey": 1441, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "egular courts. fluffily even grouches " }
+{ "l_orderkey": 1475, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31324.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1997-12-30", "l_receiptdate": "1998-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular theodolites mold across th" }
+{ "l_orderkey": 2439, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ites. furiously" }
+{ "l_orderkey": 2497, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50118.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even, regular requests across " }
+{ "l_orderkey": 2565, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43853.88, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ngly silent " }
+{ "l_orderkey": 2594, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48030.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "beans. instructions across t" }
+{ "l_orderkey": 2695, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 21926.94, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. furiously ironic platelets ar" }
+{ "l_orderkey": 2789, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "d the carefully iron" }
+{ "l_orderkey": 3203, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24015.22, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uses. fluffily ironic pinto bea" }
+{ "l_orderkey": 3361, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6264.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages sleep. furiously unus" }
+{ "l_orderkey": 3457, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 46986.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-12", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages. care" }
+{ "l_orderkey": 3521, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "onic dependencies haggle. fur" }
+{ "l_orderkey": 3618, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50118.72, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tructions atop the ironi" }
+{ "l_orderkey": 3648, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-14", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s nag packages." }
+{ "l_orderkey": 3746, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29235.92, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s after the even, special requests" }
+{ "l_orderkey": 4065, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11485.54, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hang silently about " }
+{ "l_orderkey": 4485, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 44898.02, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-28", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". blithely" }
+{ "l_orderkey": 4518, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9397.26, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-07-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " pending deposits. slyly re" }
+{ "l_orderkey": 4675, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12529.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits affix carefully" }
+{ "l_orderkey": 4773, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24015.22, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-01-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly express grouches wak" }
+{ "l_orderkey": 4931, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20882.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-25", "l_commitdate": "1994-12-21", "l_receiptdate": "1995-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furious" }
+{ "l_orderkey": 4966, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12529.68, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "eodolites. ironic requests across the exp" }
+{ "l_orderkey": 4992, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17750.38, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s along the perma" }
+{ "l_orderkey": 4996, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13573.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-17", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "o beans use about the furious" }
+{ "l_orderkey": 5154, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15662.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-23", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "even packages. packages use" }
+{ "l_orderkey": 5415, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48030.44, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the fluffily " }
+{ "l_orderkey": 5441, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34456.62, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-10-06", "l_receiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ges. final instruction" }
+{ "l_orderkey": 5442, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "have to sleep furiously bold ideas. blith" }
+{ "l_orderkey": 5826, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4176.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages across the fluffily spec" }
+{ "l_orderkey": 134, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 49121.58, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-16", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s! carefully unusual requests boost careful" }
+{ "l_orderkey": 583, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1045.14, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " regular, regular ideas. even, bra" }
+{ "l_orderkey": 834, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37625.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-28", "l_commitdate": "1994-07-25", "l_receiptdate": "1994-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ccounts haggle after the furiously " }
+{ "l_orderkey": 930, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10451.4, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-02-17", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "blithely bold i" }
+{ "l_orderkey": 1762, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25083.36, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts solve alongside of the fluffily " }
+{ "l_orderkey": 1797, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16722.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-06-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans wake regular accounts. blit" }
+{ "l_orderkey": 2118, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1996-12-20", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y ironic accounts sleep upon the packages. " }
+{ "l_orderkey": 2531, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48076.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-27", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e final, bold pains. ir" }
+{ "l_orderkey": 3109, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51211.86, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even pearls. furiously pending " }
+{ "l_orderkey": 3554, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18812.52, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle. furiously fluffy requests ac" }
+{ "l_orderkey": 3653, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39715.32, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-05-13", "l_receiptdate": "1994-07-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ainst the " }
+{ "l_orderkey": 3687, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33444.48, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deas cajole fo" }
+{ "l_orderkey": 3812, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34489.62, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-10", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "posits engage. ironic, regular p" }
+{ "l_orderkey": 3873, "l_partkey": 145, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45986.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly even platelets wake. " }
+{ "l_orderkey": 3877, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49121.58, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-08-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "furiously quick requests nag along the theo" }
+{ "l_orderkey": 3907, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 42850.74, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-25", "l_commitdate": "1992-10-17", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s above the unusual ideas sleep furiousl" }
+{ "l_orderkey": 4198, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13586.82, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-18", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furious excuses. bli" }
+{ "l_orderkey": 4327, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-27", "l_receiptdate": "1995-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic dolphins" }
+{ "l_orderkey": 4512, "l_partkey": 145, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 21947.94, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-12-30", "l_receiptdate": "1995-11-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lly unusual pinto b" }
+{ "l_orderkey": 4711, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15677.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " beans wake. deposits could bo" }
+{ "l_orderkey": 4807, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35534.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas. deposits according to the fin" }
+{ "l_orderkey": 4998, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 25083.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-25", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " unwind about" }
+{ "l_orderkey": 5667, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38670.18, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-09-17", "l_receiptdate": "1995-10-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s cajole blit" }
+{ "l_orderkey": 5954, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20902.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-03-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ke furiously blithely special packa" }
+{ "l_orderkey": 7, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. instructions" }
+{ "l_orderkey": 194, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 37661.04, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "pecial packages wake after the slyly r" }
+{ "l_orderkey": 610, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40799.46, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "counts. ironic warhorses are " }
+{ "l_orderkey": 678, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20922.8, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "furiously express excuses. foxes eat fu" }
+{ "l_orderkey": 711, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2092.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely across t" }
+{ "l_orderkey": 993, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34522.62, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-10-24", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fluffily. quiet excuses sleep furiously sly" }
+{ "l_orderkey": 1286, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 42891.74, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-02", "l_commitdate": "1993-08-06", "l_receiptdate": "1993-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " the furiously expre" }
+{ "l_orderkey": 1733, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13599.82, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-03", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-08-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites sleep furious" }
+{ "l_orderkey": 2215, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20922.8, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " unusual deposits haggle carefully. ide" }
+{ "l_orderkey": 2406, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35568.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hinly even accounts are slyly q" }
+{ "l_orderkey": 2722, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15692.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-02", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully final asympt" }
+{ "l_orderkey": 2884, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26153.5, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1997-12-06", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "onic theodolites with the instructi" }
+{ "l_orderkey": 3041, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-08-14", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "iously across the silent pinto beans. furi" }
+{ "l_orderkey": 3394, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44984.02, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "hockey players. slyly regular requests afte" }
+{ "l_orderkey": 3584, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11507.54, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely slyly " }
+{ "l_orderkey": 3714, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14645.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ending ideas. thinly unusual theodo" }
+{ "l_orderkey": 4198, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50214.72, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-09-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole carefully final, ironic ide" }
+{ "l_orderkey": 4676, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4184.56, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "detect above the ironic platelets. fluffily" }
+{ "l_orderkey": 4772, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16738.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "egular accounts wake s" }
+{ "l_orderkey": 4960, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 38707.18, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ending theodolites w" }
+{ "l_orderkey": 5185, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 52307.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-19", "l_receiptdate": "1997-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "final platelets. ideas sleep careful" }
+{ "l_orderkey": 5249, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30338.06, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-16", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " players. f" }
+{ "l_orderkey": 5285, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1046.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing deposits integra" }
+{ "l_orderkey": 5345, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2092.28, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-18", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the slyly specia" }
+{ "l_orderkey": 5798, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 41845.6, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " integrate carefu" }
+{ "l_orderkey": 5895, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32430.34, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final deposits nod slyly careful" }
+{ "l_orderkey": 5921, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26153.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-19", "l_commitdate": "1994-06-15", "l_receiptdate": "1994-06-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nd the slyly regular deposits. quick" }
+{ "l_orderkey": 225, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25131.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-10-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ironic accounts are final account" }
+{ "l_orderkey": 257, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7329.98, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ackages sleep bold realms. f" }
+{ "l_orderkey": 258, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 37697.04, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nic asymptotes. slyly silent r" }
+{ "l_orderkey": 614, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14659.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-14", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ular packages haggle about the pack" }
+{ "l_orderkey": 931, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 50262.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep alongside of the fluffy " }
+{ "l_orderkey": 1122, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26178.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "d furiously. pinto " }
+{ "l_orderkey": 1126, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 14659.96, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nstructions. blithe" }
+{ "l_orderkey": 1155, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24084.22, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly unusual packages. iro" }
+{ "l_orderkey": 1184, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4188.56, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " express packages. slyly expres" }
+{ "l_orderkey": 1600, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31414.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-03", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al escapades alongside of the depo" }
+{ "l_orderkey": 1763, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13612.82, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1997-01-24", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s sleep carefully. fluffily unusua" }
+{ "l_orderkey": 2016, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2094.28, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "carefully according to the " }
+{ "l_orderkey": 2117, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3141.42, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tes cajole" }
+{ "l_orderkey": 2279, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12565.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccounts. slyl" }
+{ "l_orderkey": 2404, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37697.04, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-05-16", "l_receiptdate": "1997-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s nag furi" }
+{ "l_orderkey": 2658, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 28272.78, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-09-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ecial packages use abov" }
+{ "l_orderkey": 2724, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21989.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-15", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "as. carefully regular dependencies wak" }
+{ "l_orderkey": 3073, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40838.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar excuses across the furiously even " }
+{ "l_orderkey": 3492, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31414.2, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-01-02", "l_receiptdate": "1995-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " detect furiously permanent, unusual accou" }
+{ "l_orderkey": 3748, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 21989.94, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fix carefully furiously express ideas. furi" }
+{ "l_orderkey": 4227, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 51309.86, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-19", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts sleep blithely carefully unusual ideas." }
+{ "l_orderkey": 4321, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34555.62, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "yly special excuses. fluffily " }
+{ "l_orderkey": 4453, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 42932.74, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "anent theodolites are slyly except t" }
+{ "l_orderkey": 4610, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30367.06, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " foxes. special, express package" }
+{ "l_orderkey": 4647, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28272.78, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ully even ti" }
+{ "l_orderkey": 4992, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49215.58, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-04", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "atterns use fluffily." }
+{ "l_orderkey": 5380, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10471.4, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1998-01-10", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully pending deposits. special, even t" }
+{ "l_orderkey": 5543, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8377.12, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-11-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "totes? iron" }
+{ "l_orderkey": 5569, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19895.66, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-30", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " detect ca" }
+{ "l_orderkey": 5954, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8377.12, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unusual th" }
+{ "l_orderkey": 5959, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17801.38, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. blithely ex" }
+{ "l_orderkey": 164, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-26", "l_commitdate": "1993-01-03", "l_receiptdate": "1992-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y carefully regular dep" }
+{ "l_orderkey": 353, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 30396.06, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions impr" }
+{ "l_orderkey": 515, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39829.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-19", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ays. furiously express requests haggle furi" }
+{ "l_orderkey": 677, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1994-01-14", "l_receiptdate": "1993-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. regular " }
+{ "l_orderkey": 774, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35636.76, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar excuses are furiously final instr" }
+{ "l_orderkey": 967, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the slyly even ideas. carefully even" }
+{ "l_orderkey": 1154, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 52407.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ove the furiously bold Tires" }
+{ "l_orderkey": 1408, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30396.06, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-02-14", "l_receiptdate": "1998-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "en accounts grow furiousl" }
+{ "l_orderkey": 1408, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20962.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-01-25", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " blithely fluffi" }
+{ "l_orderkey": 1508, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s the blithely bold instruction" }
+{ "l_orderkey": 1632, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14673.96, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-15", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-01-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oxes. deposits nag slyly along the slyly " }
+{ "l_orderkey": 1825, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40877.46, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ual, bold ideas haggle above the quickly ir" }
+{ "l_orderkey": 1893, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes bo" }
+{ "l_orderkey": 2465, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47166.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y silent foxes. final pinto beans above " }
+{ "l_orderkey": 2531, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-27", "l_commitdate": "1996-07-03", "l_receiptdate": "1996-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t the dogged, un" }
+{ "l_orderkey": 2562, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-10-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " slyly final ideas haggle car" }
+{ "l_orderkey": 2562, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 38781.18, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-29", "l_commitdate": "1992-10-06", "l_receiptdate": "1992-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ". slyly regular ideas according to the fl" }
+{ "l_orderkey": 2566, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19914.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-11-24", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests. silent" }
+{ "l_orderkey": 2598, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41925.6, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "the enticing" }
+{ "l_orderkey": 2753, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 20962.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " express pack" }
+{ "l_orderkey": 2757, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27251.64, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-10-02", "l_receiptdate": "1995-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "around the blithely" }
+{ "l_orderkey": 2790, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11529.54, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-28", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "lar requests poach slyly foxes" }
+{ "l_orderkey": 2851, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-12", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-12-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y special theodolites. carefully" }
+{ "l_orderkey": 3111, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42973.74, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-12-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily slow ideas. " }
+{ "l_orderkey": 3172, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " final packages. " }
+{ "l_orderkey": 3425, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25155.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ajole blithely sl" }
+{ "l_orderkey": 3712, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39829.32, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-15", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s nag carefully-- even, reg" }
+{ "l_orderkey": 3840, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42973.74, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " nag slyly? slyly pending accounts " }
+{ "l_orderkey": 3877, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely about the dogged ideas. ac" }
+{ "l_orderkey": 3908, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r instructions was requests. ironically " }
+{ "l_orderkey": 4161, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19914.66, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "beans breach s" }
+{ "l_orderkey": 4547, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15722.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic gifts integrate " }
+{ "l_orderkey": 4838, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2096.28, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-09-16", "l_receiptdate": "1992-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "hely final notornis are furiously blithe" }
+{ "l_orderkey": 4934, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-04-09", "l_receiptdate": "1997-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle alongside of the" }
+{ "l_orderkey": 4964, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 48214.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "althy deposits" }
+{ "l_orderkey": 4995, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-01", "l_receiptdate": "1996-04-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "t blithely. requests affix blithely. " }
+{ "l_orderkey": 5156, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly even orbi" }
+{ "l_orderkey": 5188, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "r attainments are across the " }
+{ "l_orderkey": 5601, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12577.68, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ep carefully a" }
+{ "l_orderkey": 5760, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l accounts among the carefully even de" }
+{ "l_orderkey": 5793, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly enticing excuses use slyly abov" }
+{ "l_orderkey": 5858, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7336.98, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". doggedly regular packages use pendin" }
+{ "l_orderkey": 5892, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7336.98, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-26", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously. quickly even deposits da" }
+{ "l_orderkey": 194, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16786.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-05-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y regular requests. furious" }
+{ "l_orderkey": 198, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15737.1, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. quickly pending deposits s" }
+{ "l_orderkey": 387, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33572.48, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gle. silent, fur" }
+{ "l_orderkey": 419, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 17835.38, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-20", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar dependencies: carefully regu" }
+{ "l_orderkey": 1380, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6294.84, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e foxes. slyly specia" }
+{ "l_orderkey": 1415, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26228.5, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-07-12", "l_receiptdate": "1994-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ect never fluff" }
+{ "l_orderkey": 1479, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34621.62, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully special courts affix. fluff" }
+{ "l_orderkey": 1638, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31474.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s cajole boldly bold requests. closely " }
+{ "l_orderkey": 1671, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22031.94, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-28", "l_commitdate": "1996-09-28", "l_receiptdate": "1996-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s accounts slee" }
+{ "l_orderkey": 2081, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13638.82, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fter the even deposi" }
+{ "l_orderkey": 2306, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "f the slyly unusual accounts. furiousl" }
+{ "l_orderkey": 2368, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-03", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ng the doggedly ironic requests are blithe" }
+{ "l_orderkey": 2438, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely; blithely special pinto beans breach" }
+{ "l_orderkey": 2688, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 44063.88, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-04-04", "l_receiptdate": "1992-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lly even account" }
+{ "l_orderkey": 2724, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30425.06, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-10", "l_commitdate": "1994-11-17", "l_receiptdate": "1995-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "l requests hagg" }
+{ "l_orderkey": 2754, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-05-15", "l_receiptdate": "1994-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely silent requests. regular depo" }
+{ "l_orderkey": 3107, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16786.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-10-20", "l_receiptdate": "1997-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular pinto beans. ironic ideas haggle" }
+{ "l_orderkey": 3296, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32523.34, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-26", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ainst the furi" }
+{ "l_orderkey": 3298, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-05-24", "l_receiptdate": "1996-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly final accou" }
+{ "l_orderkey": 3300, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he fluffily final a" }
+{ "l_orderkey": 4514, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12589.68, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " carefully ironic foxes nag caref" }
+{ "l_orderkey": 4546, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10491.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-16", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "above the enticingly ironic dependencies" }
+{ "l_orderkey": 4832, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1998-02-12", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ages. slyly express deposits cajole car" }
+{ "l_orderkey": 4871, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 36719.9, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-11", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ackages sle" }
+{ "l_orderkey": 4928, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35670.76, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-12", "l_commitdate": "1993-12-31", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", regular depos" }
+{ "l_orderkey": 4960, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "e blithely carefully fina" }
+{ "l_orderkey": 5157, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41965.6, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-08-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ial packages according to " }
+{ "l_orderkey": 5346, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22031.94, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-11", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "integrate blithely a" }
+{ "l_orderkey": 5382, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3147.42, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully unusua" }
+{ "l_orderkey": 5445, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10491.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-10-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ncies abou" }
+{ "l_orderkey": 5766, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-12-07", "l_receiptdate": "1993-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " furiously unusual courts. slyly final pear" }
+{ "l_orderkey": 5798, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7343.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-06-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts against the blithely final p" }
+{ "l_orderkey": 5958, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34621.62, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lar, regular accounts wake furi" }
+{ "l_orderkey": 485, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52507.5, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-28", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "iously quick excuses. carefully final f" }
+{ "l_orderkey": 677, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26253.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-12", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " packages integrate blithely" }
+{ "l_orderkey": 1025, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37805.4, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e unusual, regular instr" }
+{ "l_orderkey": 1251, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7351.05, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously pe" }
+{ "l_orderkey": 1317, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37805.4, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-03", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits. quic" }
+{ "l_orderkey": 1701, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49357.05, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-06-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly final requests cajole requests. f" }
+{ "l_orderkey": 1829, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12601.8, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-23", "l_commitdate": "1994-07-13", "l_receiptdate": "1994-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ges wake furiously express pinto" }
+{ "l_orderkey": 1856, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23103.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-05-26", "l_receiptdate": "1992-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets detect slyly regular packages. ca" }
+{ "l_orderkey": 2499, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15752.25, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-12-06", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " slyly across the slyly" }
+{ "l_orderkey": 2532, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-26", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "er the slyly pending" }
+{ "l_orderkey": 2561, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2100.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-14", "l_commitdate": "1998-01-21", "l_receiptdate": "1998-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s are. silently silent foxes sleep about" }
+{ "l_orderkey": 3333, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28354.05, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-06", "l_commitdate": "1992-10-26", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s dazzle fluffil" }
+{ "l_orderkey": 4038, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30454.35, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-07", "l_commitdate": "1996-03-08", "l_receiptdate": "1996-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ffix. quietly ironic packages a" }
+{ "l_orderkey": 4192, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46206.6, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "structions mai" }
+{ "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" }
+{ "l_orderkey": 4550, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9451.35, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l dependencies boost slyly after th" }
+{ "l_orderkey": 4674, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52507.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-13", "l_commitdate": "1994-06-15", "l_receiptdate": "1994-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "haggle about the blithel" }
+{ "l_orderkey": 4711, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23103.3, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-06-18", "l_receiptdate": "1998-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "along the quickly careful packages. bli" }
+{ "l_orderkey": 4773, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely final deposits nag after t" }
+{ "l_orderkey": 4805, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7351.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " requests. regular deposit" }
+{ "l_orderkey": 4864, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29404.2, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "thely around the bli" }
+{ "l_orderkey": 4931, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26253.75, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-01-05", "l_receiptdate": "1994-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "aggle bravely according to the quic" }
+{ "l_orderkey": 5157, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27303.9, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nto beans cajole car" }
+{ "l_orderkey": 5253, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39905.7, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "onic dependencies are furiou" }
+{ "l_orderkey": 5319, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32554.65, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-26", "l_commitdate": "1996-03-07", "l_receiptdate": "1996-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "d carefully about the courts. fluffily spe" }
+{ "l_orderkey": 5444, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42006.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even packages." }
+{ "l_orderkey": 5537, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15752.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits. permanently pending packag" }
+{ "l_orderkey": 5734, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6300.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-27", "l_commitdate": "1997-12-19", "l_receiptdate": "1997-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s. regular platelets cajole furiously. regu" }
+{ "l_orderkey": 5892, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38855.55, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-12", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "maintain. bold, expre" }
+{ "l_orderkey": 224, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16818.4, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y unusual foxes " }
+{ "l_orderkey": 519, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3153.45, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1998-01-25", "l_receiptdate": "1998-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "erve blithely blithely ironic asymp" }
+{ "l_orderkey": 579, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9460.35, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-20", "l_commitdate": "1998-04-28", "l_receiptdate": "1998-07-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e ironic, express deposits are furiously" }
+{ "l_orderkey": 773, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quickly eve" }
+{ "l_orderkey": 1061, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es are slyly expr" }
+{ "l_orderkey": 1089, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49404.05, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-26", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "aggle furiously among the bravely eve" }
+{ "l_orderkey": 1122, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15767.25, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-15", "l_commitdate": "1997-03-15", "l_receiptdate": "1997-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olve blithely regular, " }
+{ "l_orderkey": 1248, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38892.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-26", "l_commitdate": "1992-02-05", "l_receiptdate": "1992-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": ". final requests integrate quickly. blit" }
+{ "l_orderkey": 1664, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10511.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-10", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-05-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "instructions up the acc" }
+{ "l_orderkey": 2727, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3153.45, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the carefully regular foxes u" }
+{ "l_orderkey": 2822, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-11", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-09-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kly about the sly" }
+{ "l_orderkey": 3014, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 50455.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1993-01-08", "l_receiptdate": "1992-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y pending theodolites wake. reg" }
+{ "l_orderkey": 3365, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38892.55, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "requests. quickly pending instructions a" }
+{ "l_orderkey": 3462, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4204.6, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ackages. fu" }
+{ "l_orderkey": 3587, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37841.4, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ully regular excuse" }
+{ "l_orderkey": 3969, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22074.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-16", "l_receiptdate": "1997-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts doze quickly final reque" }
+{ "l_orderkey": 4256, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23125.3, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ", final platelets are slyly final pint" }
+{ "l_orderkey": 4454, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21023.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lar theodolites. even instructio" }
+{ "l_orderkey": 4931, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8409.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts boost. packages wake sly" }
+{ "l_orderkey": 5093, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32585.65, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-09-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " against the" }
+{ "l_orderkey": 5222, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1051.15, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-19", "l_commitdate": "1994-07-16", "l_receiptdate": "1994-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "idle requests. carefully pending pinto bean" }
+{ "l_orderkey": 5537, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-11-08", "l_receiptdate": "1997-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly bold packages are. qu" }
+{ "l_orderkey": 5605, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-13", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lowly special courts nag among the furi" }
+{ "l_orderkey": 5730, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2102.3, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ely ironic foxes. carefu" }
+{ "l_orderkey": 7, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 39981.7, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns haggle carefully ironic deposits. bl" }
+{ "l_orderkey": 422, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26303.75, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "carefully bold theodolit" }
+{ "l_orderkey": 449, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. blithely ironic " }
+{ "l_orderkey": 896, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11573.65, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "the multipliers sleep" }
+{ "l_orderkey": 928, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 48398.9, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-04-09", "l_receiptdate": "1995-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " beans sleep against the carefully ir" }
+{ "l_orderkey": 962, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-06-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "across the furiously regular escapades daz" }
+{ "l_orderkey": 1281, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13677.95, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully final platelets wa" }
+{ "l_orderkey": 1732, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9469.35, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-01-29", "l_receiptdate": "1994-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular platelets. deposits wak" }
+{ "l_orderkey": 1889, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43138.15, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-07-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s! furiously pending r" }
+{ "l_orderkey": 1954, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32616.65, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "against the packages. bold, ironic e" }
+{ "l_orderkey": 2050, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50503.2, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-10-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " final packages. pinto" }
+{ "l_orderkey": 2368, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16834.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-31", "l_commitdate": "1993-10-22", "l_receiptdate": "1993-11-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "telets wake carefully iro" }
+{ "l_orderkey": 2402, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25251.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-21", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "as; blithely ironic requ" }
+{ "l_orderkey": 2403, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19990.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. ironic in" }
+{ "l_orderkey": 3207, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17886.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eep against the instructions. gifts hag" }
+{ "l_orderkey": 3619, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 45242.45, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " bold, even" }
+{ "l_orderkey": 3750, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 34720.95, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-27", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "theodolites haggle. slyly pendin" }
+{ "l_orderkey": 3841, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42086.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its. quickly regular ideas nag carefully" }
+{ "l_orderkey": 3878, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21043.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-13", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "about the carefully ironic pa" }
+{ "l_orderkey": 4230, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18938.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " the final acco" }
+{ "l_orderkey": 4327, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-28", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-05-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "arefully sile" }
+{ "l_orderkey": 4390, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 36825.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-30", "l_commitdate": "1995-07-02", "l_receiptdate": "1995-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ongside of the slyly regular ideas" }
+{ "l_orderkey": 4454, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-06", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ully. carefully final accounts accordi" }
+{ "l_orderkey": 4832, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1998-02-01", "l_receiptdate": "1998-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. blithely bold pinto beans should have" }
+{ "l_orderkey": 4871, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s integrate after the a" }
+{ "l_orderkey": 5159, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he furiously sile" }
+{ "l_orderkey": 5765, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48398.9, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ccounts sleep about th" }
+{ "l_orderkey": 322, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12637.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular theodolites promise qu" }
+{ "l_orderkey": 386, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 41072.85, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-02-28", "l_receiptdate": "1995-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "hely. carefully regular accounts hag" }
+{ "l_orderkey": 387, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lithely final theodolites." }
+{ "l_orderkey": 419, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-12-25", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y above the bli" }
+{ "l_orderkey": 450, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ily carefully final depo" }
+{ "l_orderkey": 548, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33700.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1994-11-20", "l_receiptdate": "1994-12-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c instruction" }
+{ "l_orderkey": 647, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15797.25, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ve the even, bold foxes sleep " }
+{ "l_orderkey": 1092, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1053.15, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-04-21", "l_receiptdate": "1995-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lent, pending requests-- requests nag accor" }
+{ "l_orderkey": 1347, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22116.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-10", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-11-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "g pinto beans affix car" }
+{ "l_orderkey": 1636, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 45285.45, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-08-10", "l_receiptdate": "1997-09-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely special r" }
+{ "l_orderkey": 1829, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14744.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-06-08", "l_receiptdate": "1994-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular deposits alongside of the flu" }
+{ "l_orderkey": 2276, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52657.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts dete" }
+{ "l_orderkey": 2694, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oxes. never iro" }
+{ "l_orderkey": 2819, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5265.75, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-29", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " fluffily unusual foxes sleep caref" }
+{ "l_orderkey": 3591, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51604.35, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " mold slyly. bl" }
+{ "l_orderkey": 3717, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47391.75, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ests wake whithout the blithely final pl" }
+{ "l_orderkey": 3782, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10531.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ven pinto b" }
+{ "l_orderkey": 3906, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "jole blithely after the furiously regular " }
+{ "l_orderkey": 3910, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1053.15, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s sleep neve" }
+{ "l_orderkey": 4354, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24222.45, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-23", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "kly along the ironic, ent" }
+{ "l_orderkey": 4359, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8425.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages affix. fluffily regular f" }
+{ "l_orderkey": 4389, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13690.95, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-06-06", "l_receiptdate": "1994-08-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nal, regula" }
+{ "l_orderkey": 4422, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4212.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "cies along the bo" }
+{ "l_orderkey": 4455, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49498.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1993-12-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. even, even accou" }
+{ "l_orderkey": 4740, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25275.6, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-09-27", "l_receiptdate": "1996-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "hely regular deposits" }
+{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38966.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ts. pinto beans use according to th" }
+{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-14", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "onic epitaphs. f" }
+{ "l_orderkey": 4999, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-08-15", "l_receiptdate": "1993-08-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ades cajole carefully unusual ide" }
+{ "l_orderkey": 5184, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-10-16", "l_receiptdate": "1998-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully express asympto" }
+{ "l_orderkey": 5382, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-22", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular accounts. even accounts integrate" }
+{ "l_orderkey": 5415, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11584.65, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-08-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts maintain carefully unusual" }
+{ "l_orderkey": 5767, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-06-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sits among the" }
+{ "l_orderkey": 5856, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 41072.85, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uickly quickly fluffy in" }
+{ "l_orderkey": 5859, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 36860.25, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-06-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular acco" }
+{ "l_orderkey": 5958, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1995-10-19", "l_receiptdate": "1996-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "n accounts. final, ironic packages " }
+{ "l_orderkey": 193, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15812.25, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-10-09", "l_receiptdate": "1993-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ffily. regular packages d" }
+{ "l_orderkey": 292, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8433.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-18", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sily bold deposits alongside of the ex" }
+{ "l_orderkey": 608, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20028.85, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ideas. the" }
+{ "l_orderkey": 1281, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40057.7, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-28", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " ideas-- blithely regular" }
+{ "l_orderkey": 1377, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5270.75, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-06-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " final, final grouches. accoun" }
+{ "l_orderkey": 1377, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-20", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s must have to mold b" }
+{ "l_orderkey": 1573, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 31624.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-29", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". blithely even theodolites boos" }
+{ "l_orderkey": 1827, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50599.2, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-09-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oxes. special, final asymptote" }
+{ "l_orderkey": 2145, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6324.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-11-29", "l_receiptdate": "1992-10-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s. fluffily express accounts sleep. slyl" }
+{ "l_orderkey": 2626, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42166.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-12-03", "l_receiptdate": "1995-10-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eans. ironic deposits haggle. depo" }
+{ "l_orderkey": 2849, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16866.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular requ" }
+{ "l_orderkey": 2852, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29516.2, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-08", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "e accounts. caref" }
+{ "l_orderkey": 3168, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1054.15, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-03-12", "l_receiptdate": "1992-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pinto beans. slyly regular courts haggle " }
+{ "l_orderkey": 3233, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6324.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-06", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "requests are quickly above the slyly p" }
+{ "l_orderkey": 3491, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29516.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-29", "l_commitdate": "1998-09-08", "l_receiptdate": "1998-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts. sly" }
+{ "l_orderkey": 3747, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14758.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-03", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "packages cajole carefu" }
+{ "l_orderkey": 3937, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 52707.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ong the carefully exp" }
+{ "l_orderkey": 3970, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10541.5, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-07-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " special packages wake after the final br" }
+{ "l_orderkey": 4032, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24245.45, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ording to the " }
+{ "l_orderkey": 4324, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 48490.9, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-03", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-11-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ular, final theodo" }
+{ "l_orderkey": 4515, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28462.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " against the even re" }
+{ "l_orderkey": 4644, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47436.75, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-04-08", "l_receiptdate": "1998-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " carefully a" }
+{ "l_orderkey": 4805, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46382.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eposits sleep furiously qui" }
+{ "l_orderkey": 4998, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12649.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-20", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " sleep slyly furiously final accounts. ins" }
+{ "l_orderkey": 5029, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-04-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! packages boost blithely. furious" }
+{ "l_orderkey": 5031, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4216.6, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "after the even frays: ironic, unusual th" }
+{ "l_orderkey": 5538, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44274.3, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "vely ironic accounts. furiously unusual acc" }
+{ "l_orderkey": 5571, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33732.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-01-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " the blithely even packages nag q" }
+{ "l_orderkey": 5606, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3162.45, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-04", "l_receiptdate": "1997-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " sauternes. asympto" }
+{ "l_orderkey": 5858, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7379.05, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-14", "l_commitdate": "1992-10-01", "l_receiptdate": "1992-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dly pending ac" }
+{ "l_orderkey": 707, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " dependencies" }
+{ "l_orderkey": 807, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51702.35, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y regular requests haggle." }
+{ "l_orderkey": 1444, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
+{ "l_orderkey": 1542, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ial instructions. ironically" }
+{ "l_orderkey": 1638, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckages are carefully even instru" }
+{ "l_orderkey": 1664, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-06", "l_commitdate": "1996-05-16", "l_receiptdate": "1996-03-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular ide" }
+{ "l_orderkey": 1734, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40095.7, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-08-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ts doubt b" }
+{ "l_orderkey": 1956, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " wake after the " }
+{ "l_orderkey": 2273, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "furiously above the ironic requests. " }
+{ "l_orderkey": 2305, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 27433.9, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "arefully final theodo" }
+{ "l_orderkey": 2436, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50647.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-22", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he furiously " }
+{ "l_orderkey": 2466, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages detect carefully: ironically sl" }
+{ "l_orderkey": 3394, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34819.95, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-17", "l_receiptdate": "1996-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ideas alongside of th" }
+{ "l_orderkey": 3651, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25323.6, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-22", "l_commitdate": "1998-07-17", "l_receiptdate": "1998-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "excuses haggle according to th" }
+{ "l_orderkey": 3808, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30599.35, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-22", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " deposits across the pac" }
+{ "l_orderkey": 4004, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9496.35, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic requests. quickly pending ide" }
+{ "l_orderkey": 4070, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42206.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "about the sentiments. quick" }
+{ "l_orderkey": 4742, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 30599.35, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-15", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "integrate closely among t" }
+{ "l_orderkey": 5350, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7386.05, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-19", "l_commitdate": "1993-12-28", "l_receiptdate": "1993-11-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "alongside of th" }
+{ "l_orderkey": 5378, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 41150.85, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-22", "l_receiptdate": "1992-12-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts are quickly around the" }
+{ "l_orderkey": 5505, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10551.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously special asym" }
+{ "l_orderkey": 5698, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47481.75, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-23", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng excuses. slyly express asymptotes" }
+{ "l_orderkey": 5956, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10551.5, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-04", "l_receiptdate": "1998-08-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ic packages am" }
+{ "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }
+{ "l_orderkey": 165, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 28516.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-03-04", "l_receiptdate": "1993-05-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "around the ironic, even orb" }
+{ "l_orderkey": 197, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. careful" }
+{ "l_orderkey": 229, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34852.95, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-04-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " deposits; bold, ruthless theodolites" }
+{ "l_orderkey": 260, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52807.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c deposits " }
+{ "l_orderkey": 517, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " slyly. express requests ar" }
+{ "l_orderkey": 578, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42246.0, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-10", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "usly even platel" }
+{ "l_orderkey": 1027, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "oxes. carefully regular deposits" }
+{ "l_orderkey": 1095, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13729.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ously even accounts. slyly bold a" }
+{ "l_orderkey": 1248, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51751.35, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "beans run quickly according to the carefu" }
+{ "l_orderkey": 1378, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9505.35, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully. carefully iron" }
+{ "l_orderkey": 1412, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21123.0, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-04", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "odolites sleep ironically" }
+{ "l_orderkey": 1700, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51751.35, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-26", "l_commitdate": "1996-07-28", "l_receiptdate": "1996-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kly even dependencies haggle fluffi" }
+{ "l_orderkey": 1735, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-14", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously after the " }
+{ "l_orderkey": 1825, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts breach fluffily spe" }
+{ "l_orderkey": 2086, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34852.95, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " slyly regular foxes. un" }
+{ "l_orderkey": 2086, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7393.05, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1994-12-10", "l_receiptdate": "1995-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " beans haggle car" }
+{ "l_orderkey": 2368, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-09-27", "l_receiptdate": "1993-10-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fily. slyly final ideas alongside o" }
+{ "l_orderkey": 2435, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23235.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e final, final deposits. carefully regular" }
+{ "l_orderkey": 2786, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43302.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ix requests. bold requests a" }
+{ "l_orderkey": 2791, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-20", "l_receiptdate": "1995-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ilent forges. quickly special pinto beans " }
+{ "l_orderkey": 3015, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7393.05, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " after the evenly special packages ca" }
+{ "l_orderkey": 3364, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10561.5, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-10", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the accounts. final, busy accounts wi" }
+{ "l_orderkey": 3488, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19010.7, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-03-19", "l_receiptdate": "1995-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s the carefully r" }
+{ "l_orderkey": 3492, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3168.45, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "the deposits. carefully " }
+{ "l_orderkey": 3942, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26403.75, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "d the quick packages" }
+{ "l_orderkey": 3968, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-14", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly regular accounts" }
+{ "l_orderkey": 4196, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31684.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular foxes us" }
+{ "l_orderkey": 4640, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y regular instructions doze furiously. reg" }
+{ "l_orderkey": 4741, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "even requests." }
+{ "l_orderkey": 4742, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33796.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-06-12", "l_receiptdate": "1995-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits boost blithely. carefully regular a" }
+{ "l_orderkey": 4994, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38021.4, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ess ideas. blithely silent brai" }
+{ "l_orderkey": 4995, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23235.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-12", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s wake furious, express dependencies." }
+{ "l_orderkey": 4996, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41189.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "equests are carefully final" }
+{ "l_orderkey": 5093, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 39077.55, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "courts. qui" }
+{ "l_orderkey": 5348, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32740.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1997-12-09", "l_receiptdate": "1998-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "are finally" }
+{ "l_orderkey": 5349, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20066.85, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "endencies use whithout the special " }
+{ "l_orderkey": 5509, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 36965.25, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "c accounts. ca" }
+{ "l_orderkey": 5669, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2112.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-04", "l_commitdate": "1996-06-15", "l_receiptdate": "1996-08-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely excuses. slyly" }
+{ "l_orderkey": 326, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 43343.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-05", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to beans wake before the furiously re" }
+{ "l_orderkey": 455, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44400.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "around the quickly blit" }
+{ "l_orderkey": 675, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1057.15, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ide of the slyly regular packages. unus" }
+{ "l_orderkey": 802, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 19028.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y regular requests engage furiously final d" }
+{ "l_orderkey": 1121, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10571.5, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dencies. quickly regular theodolites n" }
+{ "l_orderkey": 1158, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24314.45, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ularly ironic requests use care" }
+{ "l_orderkey": 1382, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32771.65, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-10-15", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "hely regular dependencies. f" }
+{ "l_orderkey": 1509, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28543.05, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-08", "l_receiptdate": "1993-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely after the " }
+{ "l_orderkey": 1729, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12685.8, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-07-24", "l_receiptdate": "1992-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y pending packages detect. carefully re" }
+{ "l_orderkey": 1761, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39114.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express requests print blithely around the" }
+{ "l_orderkey": 1763, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45457.45, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits integrate blithely pending, quic" }
+{ "l_orderkey": 1863, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50743.2, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-08", "l_commitdate": "1993-11-05", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "onic theodolites alongside of the pending a" }
+{ "l_orderkey": 2146, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6342.9, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing to the requests. dependencies boost " }
+{ "l_orderkey": 2178, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15857.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l accounts. quickly expr" }
+{ "l_orderkey": 2433, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 40171.7, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". slyly regular requests sle" }
+{ "l_orderkey": 2531, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3171.45, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-08-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he quickly ev" }
+{ "l_orderkey": 2694, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37000.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-24", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "atelets past the furiously final deposits " }
+{ "l_orderkey": 3458, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2114.3, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-03-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ironic packages haggle past the furiously " }
+{ "l_orderkey": 3522, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 48628.9, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-11-30", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "d the express, silent foxes. blit" }
+{ "l_orderkey": 3553, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 38057.4, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " realms. pending, bold theodolites " }
+{ "l_orderkey": 3841, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1057.15, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-10", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " boost even re" }
+{ "l_orderkey": 4069, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52857.5, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even foxes among the express wate" }
+{ "l_orderkey": 4389, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21143.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ng the carefully express d" }
+{ "l_orderkey": 4741, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 35943.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sly special packages after the furiously" }
+{ "l_orderkey": 4770, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31714.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-25", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ffily carefully ironic ideas. ironic d" }
+{ "l_orderkey": 4869, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26428.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e according t" }
+{ "l_orderkey": 5792, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49686.05, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "regular, ironic excuses n" }
+{ "l_orderkey": 5922, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39114.55, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1996-12-16", "l_receiptdate": "1997-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s wake slyly. requests cajole furiously asy" }
+{ "l_orderkey": 7, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ithely regula" }
+{ "l_orderkey": 135, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34918.95, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ptotes boost slowly care" }
+{ "l_orderkey": 449, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23279.3, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "furiously final theodolites eat careful" }
+{ "l_orderkey": 736, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48674.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-01", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uctions cajole" }
+{ "l_orderkey": 839, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24337.45, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng ideas haggle accord" }
+{ "l_orderkey": 1317, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 27511.9, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "leep along th" }
+{ "l_orderkey": 1412, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11639.65, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-04-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se slyly. special, unusual accounts nag bl" }
+{ "l_orderkey": 1859, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "across the p" }
+{ "l_orderkey": 1955, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43384.15, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " carefully against the furiously reg" }
+{ "l_orderkey": 2144, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10581.5, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-16", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " furiously unusual ideas. carefull" }
+{ "l_orderkey": 2567, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 52907.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "pinto beans? r" }
+{ "l_orderkey": 3136, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45500.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-09-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". special theodolites ha" }
+{ "l_orderkey": 3522, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19046.7, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-16", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits wake carefully pen" }
+{ "l_orderkey": 4227, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20104.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ns sleep along the blithely even theodolit" }
+{ "l_orderkey": 4773, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6348.9, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "latelets haggle s" }
+{ "l_orderkey": 4993, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32802.65, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-10-15", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nwind thinly platelets. a" }
+{ "l_orderkey": 5089, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4232.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-09-28", "l_receiptdate": "1992-10-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nts sleep blithely " }
+{ "l_orderkey": 5249, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12697.8, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-07", "l_receiptdate": "1995-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "press depths could have to sleep carefu" }
+{ "l_orderkey": 5254, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 35977.1, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously above the furiously " }
+{ "l_orderkey": 5442, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22221.15, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ffily furiously ironic theodolites. furio" }
+{ "l_orderkey": 5665, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43384.15, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-22", "l_receiptdate": "1993-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " idle ideas across " }
+{ "l_orderkey": 5669, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42326.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-06-15", "l_receiptdate": "1996-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ar accounts alongside of the final, p" }
+{ "l_orderkey": 5794, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44442.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "he careful" }
+{ "l_orderkey": 5828, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39151.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-05-30", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully spec" }
+{ "l_orderkey": 5952, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24337.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-05-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e blithely packages. eve" }
+{ "l_orderkey": 231, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e furiously ironic pinto beans." }
+{ "l_orderkey": 325, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36011.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly bold deposits. always iron" }
+{ "l_orderkey": 519, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1059.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold requests believe furiou" }
+{ "l_orderkey": 551, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21183.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "r ideas. final, even ideas hinder alongside" }
+{ "l_orderkey": 613, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7414.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-09-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ously blithely final pinto beans. regula" }
+{ "l_orderkey": 1222, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-05", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously bold instructions" }
+{ "l_orderkey": 1315, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33892.8, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "neath the final p" }
+{ "l_orderkey": 1954, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ongside of the slyly unusual requests. reg" }
+{ "l_orderkey": 1955, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11650.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-06-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ously quickly pendi" }
+{ "l_orderkey": 2277, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32833.65, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-07", "l_commitdate": "1995-03-19", "l_receiptdate": "1995-03-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ic instructions detect ru" }
+{ "l_orderkey": 2371, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39188.55, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-11", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s boost fluffil" }
+{ "l_orderkey": 2468, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19064.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-26", "l_receiptdate": "1997-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cies. fluffily r" }
+{ "l_orderkey": 2499, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 41306.85, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-27", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "otes sublat" }
+{ "l_orderkey": 2595, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30715.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic accounts haggle carefully fin" }
+{ "l_orderkey": 3363, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1995-12-01", "l_receiptdate": "1996-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uickly bold ide" }
+{ "l_orderkey": 3588, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47661.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-07", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ecial pains integrate blithely. reques" }
+{ "l_orderkey": 3714, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ccounts cajole fu" }
+{ "l_orderkey": 3776, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14828.1, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-30", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y special ideas. express packages pr" }
+{ "l_orderkey": 3938, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48720.9, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-05-04", "l_receiptdate": "1993-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly even foxes are slyly fu" }
+{ "l_orderkey": 4006, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19064.7, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-03-08", "l_receiptdate": "1995-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "gouts! slyly iron" }
+{ "l_orderkey": 4451, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20123.85, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-11-26", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly after the fluffi" }
+{ "l_orderkey": 4549, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ding to the regular, silent requests" }
+{ "l_orderkey": 4610, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " final theodolites " }
+{ "l_orderkey": 4738, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10591.5, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-10", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "hins above the" }
+{ "l_orderkey": 4743, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3177.45, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "al requests. express idea" }
+{ "l_orderkey": 4807, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas wake bli" }
+{ "l_orderkey": 5062, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52957.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the regular, unusual pains. specia" }
+{ "l_orderkey": 5092, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 52957.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-30", "l_commitdate": "1996-01-14", "l_receiptdate": "1995-12-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r platelets maintain car" }
+{ "l_orderkey": 5409, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39188.55, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-02-10", "l_receiptdate": "1992-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ously regular packages. packages" }
+{ "l_orderkey": 5728, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42366.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-28", "l_commitdate": "1995-01-17", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "final deposits. theodolite" }
+{ "l_orderkey": 5825, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24360.45, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-04-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " special pinto beans. dependencies haggl" }
+{ "l_orderkey": 5957, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33892.8, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " boost carefully across the " }
+{ "l_orderkey": 645, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-01-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heodolites b" }
+{ "l_orderkey": 710, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 48767.36, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ges use; blithely pending excuses inte" }
+{ "l_orderkey": 769, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4240.64, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-08-12", "l_receiptdate": "1993-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ideas. even" }
+{ "l_orderkey": 806, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-08-11", "l_receiptdate": "1996-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fily pending " }
+{ "l_orderkey": 1157, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14842.24, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites. fluffily re" }
+{ "l_orderkey": 1220, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38165.76, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar packages. blithely final acc" }
+{ "l_orderkey": 1282, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20143.04, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-04-07", "l_receiptdate": "1992-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts x-ray across the furi" }
+{ "l_orderkey": 1317, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pinto beans according to the final, pend" }
+{ "l_orderkey": 1346, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the pinto " }
+{ "l_orderkey": 1409, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18022.72, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-04-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "pending accounts poach. care" }
+{ "l_orderkey": 1441, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 39225.92, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-26", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-04-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts. slyly special dolphins b" }
+{ "l_orderkey": 1888, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 4240.64, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lphins. ironically special theodolit" }
+{ "l_orderkey": 2118, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25443.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the slyly bold depende" }
+{ "l_orderkey": 2176, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-03-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ruthless deposits according to the ent" }
+{ "l_orderkey": 2211, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular, express" }
+{ "l_orderkey": 2374, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25443.84, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". requests are above t" }
+{ "l_orderkey": 2532, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-23", "l_commitdate": "1996-01-04", "l_receiptdate": "1995-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rve carefully slyly ironic accounts! fluf" }
+{ "l_orderkey": 2535, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20143.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ructions. final requests" }
+{ "l_orderkey": 2562, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "eep against the furiously r" }
+{ "l_orderkey": 2823, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19082.88, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final deposits. furiously regular foxes u" }
+{ "l_orderkey": 2854, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 21203.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "rs impress after the deposits. " }
+{ "l_orderkey": 3364, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-09", "l_commitdate": "1997-08-01", "l_receiptdate": "1997-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously regular ideas haggle furiously b" }
+{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1060.16, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final excuses. carefully even waters hagg" }
+{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11661.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual re" }
+{ "l_orderkey": 3584, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24383.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-10", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l platelets until the asymptotes " }
+{ "l_orderkey": 3939, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8481.28, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-04-05", "l_receiptdate": "1996-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e packages. express, pen" }
+{ "l_orderkey": 4135, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-05-23", "l_receiptdate": "1997-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he fluffil" }
+{ "l_orderkey": 4224, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53008.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-10", "l_receiptdate": "1997-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "side of the carefully silent dep" }
+{ "l_orderkey": 4262, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 43466.56, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cuses unwind ac" }
+{ "l_orderkey": 4325, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19082.88, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-09-28", "l_receiptdate": "1996-10-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ". blithely" }
+{ "l_orderkey": 4389, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ironic request" }
+{ "l_orderkey": 4454, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21203.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "quickly regular requests. furiously" }
+{ "l_orderkey": 4807, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es use final excuses. furiously final" }
+{ "l_orderkey": 4867, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3180.48, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "yly silent deposits" }
+{ "l_orderkey": 5125, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " thinly even pack" }
+{ "l_orderkey": 5443, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1997-01-08", "l_receiptdate": "1997-01-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "use carefully above the pinto bea" }
+{ "l_orderkey": 5504, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-13", "l_receiptdate": "1993-02-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ajole carefully. care" }
+{ "l_orderkey": 5506, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6360.96, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hely according to the furiously unusua" }
+{ "l_orderkey": 5633, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29684.48, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "as boost quickly. unusual pinto " }
+{ "l_orderkey": 5830, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y bold excuses" }
+{ "l_orderkey": 5863, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22263.36, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "atelets nag blithely furi" }
+{ "l_orderkey": 5925, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 43466.56, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " across the pending deposits nag caref" }
+{ "l_orderkey": 771, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40324.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " quickly final requests are final packages." }
+{ "l_orderkey": 898, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9550.44, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-04", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e slyly across the blithe" }
+{ "l_orderkey": 967, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-06", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ngage blith" }
+{ "l_orderkey": 1092, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29712.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "affix carefully. u" }
+{ "l_orderkey": 1121, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28651.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts cajole slyly abou" }
+{ "l_orderkey": 1315, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20162.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-05", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-08-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nal, regular warhorses about the fu" }
+{ "l_orderkey": 1383, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20162.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-24", "l_commitdate": "1993-07-07", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lyly unusual accounts sle" }
+{ "l_orderkey": 1732, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43507.56, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "quests sublate against the silent " }
+{ "l_orderkey": 1765, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38201.76, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-03-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "he blithely pending accou" }
+{ "l_orderkey": 1895, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45629.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-26", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-08-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " carefully eve" }
+{ "l_orderkey": 2240, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30773.64, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lyly even ideas w" }
+{ "l_orderkey": 2273, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21223.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "cuses. quickly enticing requests wake " }
+{ "l_orderkey": 2407, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7428.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-11", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes are carefully accordin" }
+{ "l_orderkey": 2438, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 28651.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-10-01", "l_receiptdate": "1993-08-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "inal accounts. slyly final reques" }
+{ "l_orderkey": 2593, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 46691.04, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-10-23", "l_receiptdate": "1993-09-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ents impress furiously; unusual theodoli" }
+{ "l_orderkey": 2817, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4244.64, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-04", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "n accounts wake across the fluf" }
+{ "l_orderkey": 3012, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39262.92, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-16", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uickly permanent packages sleep caref" }
+{ "l_orderkey": 3269, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42446.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "es. pending d" }
+{ "l_orderkey": 3552, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 38201.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-24", "l_receiptdate": "1997-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly regular theodolites. fin" }
+{ "l_orderkey": 3618, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27590.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-23", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "iously regular deposits cajole ruthless" }
+{ "l_orderkey": 4004, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46691.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ut the sauternes. bold, ironi" }
+{ "l_orderkey": 4391, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ong the silent deposits" }
+{ "l_orderkey": 4645, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37140.6, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sias believe bl" }
+{ "l_orderkey": 4871, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18039.72, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-09", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es. carefully ev" }
+{ "l_orderkey": 4935, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13795.08, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-08-13", "l_receiptdate": "1993-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly requests. final deposits might " }
+{ "l_orderkey": 5031, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42446.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ns hang blithely across th" }
+{ "l_orderkey": 5060, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15917.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-08-20", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ular deposits sl" }
+{ "l_orderkey": 5062, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-25", "l_receiptdate": "1992-11-05", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "furiously pending requests are ruthles" }
+{ "l_orderkey": 5063, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-10-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously special " }
+{ "l_orderkey": 5415, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11672.76, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle among t" }
+{ "l_orderkey": 5570, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39262.92, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-29", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-09-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y ironic pin" }
+{ "l_orderkey": 5858, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-12-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "al excuses. bold" }
+{ "l_orderkey": 5863, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47752.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-01-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits are ab" }
+{ "l_orderkey": 35, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36113.44, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s are carefully against the f" }
+{ "l_orderkey": 165, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45672.88, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "jole slyly according " }
+{ "l_orderkey": 192, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tes. carefu" }
+{ "l_orderkey": 258, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47797.2, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular excuses-- fluffily ruthl" }
+{ "l_orderkey": 259, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14870.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-11-20", "l_receiptdate": "1993-11-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully even, regul" }
+{ "l_orderkey": 288, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32926.96, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ns. fluffily" }
+{ "l_orderkey": 358, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33989.12, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lyly express deposits " }
+{ "l_orderkey": 420, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly against the blithely re" }
+{ "l_orderkey": 422, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26554.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-07-09", "l_receiptdate": "1997-09-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ep along the furiousl" }
+{ "l_orderkey": 450, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44610.72, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-05-29", "l_receiptdate": "1995-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y asymptotes. regular depen" }
+{ "l_orderkey": 551, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-08-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y along the carefully ex" }
+{ "l_orderkey": 833, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9559.44, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1994-04-26", "l_receiptdate": "1994-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ecial, even requests. even, bold instructi" }
+{ "l_orderkey": 1122, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25491.84, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely requests. slyly pending r" }
+{ "l_orderkey": 1382, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19118.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-10-19", "l_receiptdate": "1993-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "hely regular deposits. fluffy s" }
+{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15932.4, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "pinto beans cajole. bravely bold" }
+{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9559.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular dependencies wake. blithely final e" }
+{ "l_orderkey": 2114, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26554.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-15", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "unts. regular, express accounts wake. b" }
+{ "l_orderkey": 2273, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19118.88, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1997-01-21", "l_receiptdate": "1997-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " beans. doggedly final packages wake" }
+{ "l_orderkey": 2432, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8497.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-16", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s about the bold, close deposit" }
+{ "l_orderkey": 2470, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31864.8, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s across the furiously fina" }
+{ "l_orderkey": 2630, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30802.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully unusual dependencies. even i" }
+{ "l_orderkey": 2691, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-06-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the even foxes. unusual theodoli" }
+{ "l_orderkey": 2723, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2124.32, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " courts boost quickly about th" }
+{ "l_orderkey": 2786, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-03", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "slow instructi" }
+{ "l_orderkey": 3527, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 53108.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-03", "l_receiptdate": "1997-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e even accounts was about th" }
+{ "l_orderkey": 3687, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20181.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly final asymptotes according to t" }
+{ "l_orderkey": 3842, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29740.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s excuses thrash carefully." }
+{ "l_orderkey": 4258, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20181.04, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly busily ironic foxes. f" }
+{ "l_orderkey": 4262, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5310.8, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-16", "l_receiptdate": "1996-10-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ironic accounts are unusu" }
+{ "l_orderkey": 4512, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31864.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly unusual package" }
+{ "l_orderkey": 4836, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans. care" }
+{ "l_orderkey": 4865, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits haggle. fur" }
+{ "l_orderkey": 4961, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43548.56, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ily against the n" }
+{ "l_orderkey": 5089, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ironic accounts" }
+{ "l_orderkey": 5346, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 37175.6, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-03-01", "l_receiptdate": "1994-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nic excuses cajole entic" }
+{ "l_orderkey": 5505, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48859.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1997-11-04", "l_receiptdate": "1998-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usly ironic dependencies haggle across " }
+{ "l_orderkey": 5510, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 49921.52, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously even requests. slyly bold accou" }
+{ "l_orderkey": 5543, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "instructions. deposits use quickly. ir" }
+{ "l_orderkey": 5635, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24429.68, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-11-10", "l_receiptdate": "1992-09-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily pending packages. bold," }
+{ "l_orderkey": 5638, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-03-27", "l_receiptdate": "1994-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press courts use f" }
+{ "l_orderkey": 5765, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32926.96, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r foxes. ev" }
+{ "l_orderkey": 5953, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5310.8, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-04-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s. blithely " }
+{ "l_orderkey": 643, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24452.68, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-09", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits are carefully according to the e" }
+{ "l_orderkey": 676, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37210.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-13", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "into beans. blithe" }
+{ "l_orderkey": 710, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49968.52, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "usual ideas into th" }
+{ "l_orderkey": 997, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-16", "l_commitdate": "1997-07-21", "l_receiptdate": "1997-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "p furiously according to t" }
+{ "l_orderkey": 1316, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8505.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages against the express requests wa" }
+{ "l_orderkey": 1795, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly. special pa" }
+{ "l_orderkey": 2246, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13821.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-15", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests. fluffily special epitaphs use" }
+{ "l_orderkey": 2502, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35084.28, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-07-22", "l_receiptdate": "1993-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "have to print" }
+{ "l_orderkey": 2789, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17010.56, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-18", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "o beans use carefully" }
+{ "l_orderkey": 2914, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-05-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cross the carefully even accounts." }
+{ "l_orderkey": 2983, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 46779.04, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-09", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly regular instruct" }
+{ "l_orderkey": 3014, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38273.76, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-20", "l_receiptdate": "1992-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ding accounts boost fu" }
+{ "l_orderkey": 3520, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37210.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-09-03", "l_receiptdate": "1997-09-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s nag carefully. sometimes unusual account" }
+{ "l_orderkey": 3652, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 41463.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-10", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-03-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y express instructions. un" }
+{ "l_orderkey": 3684, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20200.04, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-19", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly carefully pending foxes. d" }
+{ "l_orderkey": 3718, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17010.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-11", "l_commitdate": "1996-12-25", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly even accounts. blithely special acco" }
+{ "l_orderkey": 3841, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 51031.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-22", "l_receiptdate": "1994-12-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " according to the regular, " }
+{ "l_orderkey": 4064, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-31", "l_receiptdate": "1997-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly regular ideas." }
+{ "l_orderkey": 4099, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51031.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle according to the slyly f" }
+{ "l_orderkey": 4258, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9568.44, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts wake permanently after the bravely" }
+{ "l_orderkey": 4292, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 42526.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ounts according to the furiously " }
+{ "l_orderkey": 4326, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press reque" }
+{ "l_orderkey": 4578, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 21263.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-11", "l_commitdate": "1992-11-09", "l_receiptdate": "1993-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "iously pending theodolites--" }
+{ "l_orderkey": 4672, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y fluffily stealt" }
+{ "l_orderkey": 4705, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29768.48, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes wake according to the unusual plate" }
+{ "l_orderkey": 4992, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46779.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-06-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "rmanent, sly packages print slyly. regula" }
+{ "l_orderkey": 5254, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47842.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-09-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " wake. blithely silent excuse" }
+{ "l_orderkey": 5698, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " asymptotes sleep slyly above the" }
+{ "l_orderkey": 7, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29796.48, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". slyly special requests haggl" }
+{ "l_orderkey": 101, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38309.76, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tes. blithely pending dolphins x-ray f" }
+{ "l_orderkey": 323, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-04-25", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cial requests " }
+{ "l_orderkey": 801, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12769.92, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. ironic pinto b" }
+{ "l_orderkey": 930, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " excuses among the furiously express ideas " }
+{ "l_orderkey": 1060, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11705.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e regular deposits: re" }
+{ "l_orderkey": 1184, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7449.12, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly warthogs. blithely bold foxes hag" }
+{ "l_orderkey": 1248, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ter the pending pl" }
+{ "l_orderkey": 2240, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6384.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-17", "l_receiptdate": "1992-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ymptotes boost. furiously bold p" }
+{ "l_orderkey": 2372, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12769.92, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1997-12-21", "l_receiptdate": "1998-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lyly according to" }
+{ "l_orderkey": 2436, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6384.96, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-25", "l_commitdate": "1995-11-30", "l_receiptdate": "1995-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "odolites. ep" }
+{ "l_orderkey": 2439, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2128.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-06-11", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "courts boos" }
+{ "l_orderkey": 2821, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-27", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "requests. blit" }
+{ "l_orderkey": 2852, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30860.64, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-21", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-05-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironi" }
+{ "l_orderkey": 3591, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4256.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he final packages. deposits serve quick" }
+{ "l_orderkey": 3621, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-06-18", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " doubt about the bold deposits. carefully" }
+{ "l_orderkey": 3811, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25539.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-05-16", "l_receiptdate": "1998-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "deposits. slyly regular accounts cajo" }
+{ "l_orderkey": 3937, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 1064.16, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-01-08", "l_receiptdate": "1998-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "refully agains" }
+{ "l_orderkey": 4451, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42566.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-18", "l_commitdate": "1994-12-25", "l_receiptdate": "1994-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y. slyly special deposits are sly" }
+{ "l_orderkey": 4514, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-07-13", "l_receiptdate": "1994-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " even, silent foxes be" }
+{ "l_orderkey": 5092, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31924.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-08", "l_receiptdate": "1996-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ss, ironic deposits. furiously stea" }
+{ "l_orderkey": 5441, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3192.48, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "are. unusual, " }
+{ "l_orderkey": 5601, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-25", "l_commitdate": "1992-04-03", "l_receiptdate": "1992-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts-- blithely final accounts cajole. carefu" }
+{ "l_orderkey": 5633, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts. slyly regular " }
+{ "l_orderkey": 5827, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3192.48, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-09-29", "l_receiptdate": "1998-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uses eat along the furiously" }
+{ "l_orderkey": 5858, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48951.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-06", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "posits withi" }
+{ "l_orderkey": 134, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37280.6, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-23", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ajole furiously. instructio" }
+{ "l_orderkey": 357, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34085.12, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-12-29", "l_receiptdate": "1997-02-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y above the carefully final accounts" }
+{ "l_orderkey": 484, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23433.52, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es are pending instructions. furiously unu" }
+{ "l_orderkey": 518, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31954.8, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-18", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly by the packages. carefull" }
+{ "l_orderkey": 902, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-10-12", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely even accounts poach furiously i" }
+{ "l_orderkey": 1031, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29824.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly ironic accounts across the q" }
+{ "l_orderkey": 1286, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-08-09", "l_receiptdate": "1993-06-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely bo" }
+{ "l_orderkey": 1413, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 52192.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nstructions br" }
+{ "l_orderkey": 1728, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46867.04, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ide of the slyly blithe" }
+{ "l_orderkey": 2117, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38345.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ronic accounts wake" }
+{ "l_orderkey": 2151, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 52192.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. f" }
+{ "l_orderkey": 2307, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-23", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "olites haggle furiously around the " }
+{ "l_orderkey": 2371, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 23433.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-26", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y daring accounts. regular ins" }
+{ "l_orderkey": 2438, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "en theodolites w" }
+{ "l_orderkey": 2534, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-12", "l_commitdate": "1996-09-26", "l_receiptdate": "1996-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "eposits doze quickly final" }
+{ "l_orderkey": 2657, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15977.4, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-12-18", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ole carefully above the ironic ideas. b" }
+{ "l_orderkey": 2848, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8521.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-21", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". silent, final ideas sublate packages. ir" }
+{ "l_orderkey": 2979, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29824.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ing, regular pinto beans. blithel" }
+{ "l_orderkey": 3015, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-21", "l_commitdate": "1992-11-07", "l_receiptdate": "1993-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "encies haggle furious" }
+{ "l_orderkey": 3168, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11716.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-03-17", "l_receiptdate": "1992-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously furious dependenc" }
+{ "l_orderkey": 3234, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely regular f" }
+{ "l_orderkey": 3429, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-03-08", "l_receiptdate": "1997-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ites poach a" }
+{ "l_orderkey": 3623, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g to the slyly regular packa" }
+{ "l_orderkey": 3654, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48997.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usly regular foxes. furio" }
+{ "l_orderkey": 3716, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully unusual accounts. flu" }
+{ "l_orderkey": 3746, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1994-10-25", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e of the careful" }
+{ "l_orderkey": 3748, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. blithely" }
+{ "l_orderkey": 3846, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 35150.28, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-03-14", "l_receiptdate": "1998-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s instructions are. fu" }
+{ "l_orderkey": 4581, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-17", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e the blithely bold pearls ha" }
+{ "l_orderkey": 4676, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50062.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1995-10-04", "l_receiptdate": "1996-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely about the carefully special requ" }
+{ "l_orderkey": 4901, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12781.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y unusual deposits prom" }
+{ "l_orderkey": 4903, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6390.96, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "azzle quickly along the blithely final pla" }
+{ "l_orderkey": 4966, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7456.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-09", "l_receiptdate": "1997-01-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly ironic tithe" }
+{ "l_orderkey": 5061, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19172.88, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "atelets among the ca" }
+{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17042.56, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites " }
+{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33019.96, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "gular excuses. fluffily even pinto beans c" }
+{ "l_orderkey": 227, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20257.04, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1996-01-30", "l_receiptdate": "1995-12-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole furiously a" }
+{ "l_orderkey": 359, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31984.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses detect spec" }
+{ "l_orderkey": 676, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 33050.96, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ial deposits cajo" }
+{ "l_orderkey": 995, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47977.2, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-07-21", "l_receiptdate": "1995-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lar packages detect blithely above t" }
+{ "l_orderkey": 1121, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use furiously. quickly silent package" }
+{ "l_orderkey": 1698, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 15992.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final ideas. even, ironic " }
+{ "l_orderkey": 1703, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38381.76, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously express " }
+{ "l_orderkey": 1730, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43712.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-08-29", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions. unusual, even Tiresi" }
+{ "l_orderkey": 1829, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6396.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s haggle! slyl" }
+{ "l_orderkey": 1862, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39447.92, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l deposits. carefully even dep" }
+{ "l_orderkey": 2021, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20257.04, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-09-05", "l_receiptdate": "1995-08-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " above the slyly fl" }
+{ "l_orderkey": 2407, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. special deposits are closely." }
+{ "l_orderkey": 2438, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29852.48, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-05", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions. bli" }
+{ "l_orderkey": 2469, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11727.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ies wake carefully b" }
+{ "l_orderkey": 2691, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1066.16, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular instructions b" }
+{ "l_orderkey": 2913, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-09-25", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "requests doze quickly. furious" }
+{ "l_orderkey": 3079, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49043.36, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es. final, regula" }
+{ "l_orderkey": 3108, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27720.16, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-12", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " slyly slow foxes wake furious" }
+{ "l_orderkey": 3200, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28786.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "as haggle furiously against the fluff" }
+{ "l_orderkey": 3555, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11727.76, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-10-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "oost caref" }
+{ "l_orderkey": 3750, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 35183.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-15", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep blithely according to the flu" }
+{ "l_orderkey": 3777, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19190.88, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-04", "l_commitdate": "1994-05-23", "l_receiptdate": "1994-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eful packages use slyly: even deposits " }
+{ "l_orderkey": 3811, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2132.32, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly fluff" }
+{ "l_orderkey": 4231, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4264.64, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-28", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lithely even packages. " }
+{ "l_orderkey": 4258, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38381.76, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ns use alongs" }
+{ "l_orderkey": 4930, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ions haggle. furiously regular ideas use " }
+{ "l_orderkey": 4993, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-09-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " final packages at the q" }
+{ "l_orderkey": 5024, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1997-01-10", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " to the expre" }
+{ "l_orderkey": 5095, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-14", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bold theodolites wake about the expr" }
+{ "l_orderkey": 5253, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26654.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-21", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "urts. even theodoli" }
+{ "l_orderkey": 5568, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "furious ide" }
+{ "l_orderkey": 5605, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly. quickly pending sen" }
+{ "l_orderkey": 5760, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6396.96, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " shall have to cajole along the " }
+{ "l_orderkey": 166, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13873.08, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "fully above the blithely fina" }
+{ "l_orderkey": 224, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12805.92, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-12", "l_commitdate": "1994-08-29", "l_receiptdate": "1994-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uriously regular packages. slyly fina" }
+{ "l_orderkey": 385, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7470.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " special asymptote" }
+{ "l_orderkey": 579, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-25", "l_receiptdate": "1998-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "refully silent ideas cajole furious" }
+{ "l_orderkey": 614, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45887.88, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-02-22", "l_receiptdate": "1993-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " express accounts wake. slyly ironic ins" }
+{ "l_orderkey": 930, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 32014.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-02-28", "l_receiptdate": "1995-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g accounts sleep along the platelets." }
+{ "l_orderkey": 1412, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11738.76, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "en packages. regular packages dete" }
+{ "l_orderkey": 1447, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20276.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1992-12-07", "l_receiptdate": "1993-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". quickly ironic " }
+{ "l_orderkey": 1601, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6402.96, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-19", "l_commitdate": "1994-09-28", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " bold sheaves. furiously per" }
+{ "l_orderkey": 1857, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42686.4, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-03-08", "l_receiptdate": "1993-02-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "slyly close d" }
+{ "l_orderkey": 1888, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 53358.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ependencies affix blithely regular warhors" }
+{ "l_orderkey": 2151, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24544.68, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " silent dependencies about the slyl" }
+{ "l_orderkey": 2208, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19208.88, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "packages are quickly bold de" }
+{ "l_orderkey": 2241, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20276.04, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " are furiously quickl" }
+{ "l_orderkey": 2563, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29880.48, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "hely regular depe" }
+{ "l_orderkey": 2917, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the furiously silent packages. pend" }
+{ "l_orderkey": 3365, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39484.92, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-09", "l_receiptdate": "1994-11-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "oze blithely. furiously ironic theodolit" }
+{ "l_orderkey": 3457, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9604.44, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quests. foxes sleep quickly" }
+{ "l_orderkey": 3461, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25611.84, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "thely. carefully re" }
+{ "l_orderkey": 3520, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40552.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-09-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "yly final packages according to the quickl" }
+{ "l_orderkey": 3620, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17074.56, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. even, pending in" }
+{ "l_orderkey": 3783, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38417.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ites haggle among the carefully unusu" }
+{ "l_orderkey": 4326, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28813.32, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-29", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "inal packages. final asymptotes about t" }
+{ "l_orderkey": 4421, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49089.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-05-21", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "g dependenci" }
+{ "l_orderkey": 4773, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 52290.84, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-26", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final reque" }
+{ "l_orderkey": 5063, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2134.32, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-07-27", "l_receiptdate": "1997-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly regular i" }
+{ "l_orderkey": 5157, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16007.4, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-27", "l_commitdate": "1997-08-30", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "cajole. spec" }
+{ "l_orderkey": 5472, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 41619.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uriously carefully " }
+{ "l_orderkey": 5696, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44820.72, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "te furious" }
+{ "l_orderkey": 5767, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11738.76, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "instructions. carefully final accou" }
+{ "l_orderkey": 5927, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34149.12, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1997-10-27", "l_receiptdate": "1997-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "telets. carefully bold accounts was" }
+{ "l_orderkey": 98, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10681.6, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-11-08", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " carefully. quickly ironic ideas" }
+{ "l_orderkey": 131, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48067.2, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic, bold accounts. careful" }
+{ "l_orderkey": 163, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45930.88, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-19", "l_commitdate": "1997-11-19", "l_receiptdate": "1997-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al, bold dependencies wake. iron" }
+{ "l_orderkey": 194, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 22431.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "accounts detect quickly dogged " }
+{ "l_orderkey": 359, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11749.76, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-15", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "rets wake blithely. slyly final dep" }
+{ "l_orderkey": 582, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-09", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lar requests. quickly " }
+{ "l_orderkey": 677, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41658.24, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ges. furiously regular packages use " }
+{ "l_orderkey": 868, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8545.28, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-08-01", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "l deposits. blithely regular pint" }
+{ "l_orderkey": 903, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13886.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sleep along the final" }
+{ "l_orderkey": 1121, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44862.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-03-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nts are slyly special packages. f" }
+{ "l_orderkey": 1315, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lites. unusual foxes affi" }
+{ "l_orderkey": 1445, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 41658.24, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-02-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ully unusual reques" }
+{ "l_orderkey": 1475, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16022.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1997-12-17", "l_receiptdate": "1998-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "xpress requests haggle after the final, fi" }
+{ "l_orderkey": 1794, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ely fluffily ironi" }
+{ "l_orderkey": 1795, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-05-22", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "he always express accounts ca" }
+{ "l_orderkey": 1889, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5340.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-06-09", "l_receiptdate": "1997-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ording to the blithely silent r" }
+{ "l_orderkey": 2050, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17090.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-07-28", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "al accounts. closely even " }
+{ "l_orderkey": 2087, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49135.36, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ter the dolphins." }
+{ "l_orderkey": 2114, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53408.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pecial pinto bean" }
+{ "l_orderkey": 2375, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3204.48, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "slyly across the furiously e" }
+{ "l_orderkey": 2434, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52339.84, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " after the requests haggle bold, fina" }
+{ "l_orderkey": 2913, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 37385.6, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-08-21", "l_receiptdate": "1997-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. quickly even braids against" }
+{ "l_orderkey": 2978, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 4272.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-31", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ffily unusual " }
+{ "l_orderkey": 3044, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3204.48, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-27", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ecoys haggle furiously pending requests." }
+{ "l_orderkey": 3654, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11749.76, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-07-30", "l_receiptdate": "1992-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "quickly along the express, ironic req" }
+{ "l_orderkey": 3808, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46999.04, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-06-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the blithely regular foxes. even, final " }
+{ "l_orderkey": 3814, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "beans cajole quickly sl" }
+{ "l_orderkey": 4739, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8545.28, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-22", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cording to the " }
+{ "l_orderkey": 4930, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29908.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e ironic, unusual courts. regula" }
+{ "l_orderkey": 4963, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40590.08, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "tegrate daringly accou" }
+{ "l_orderkey": 5093, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42726.4, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-11-04", "l_receiptdate": "1993-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ing pinto beans. quickly bold dependenci" }
+{ "l_orderkey": 5191, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42726.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-04-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nes haggle sometimes. requests eng" }
+{ "l_orderkey": 5349, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14954.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-17", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully regular " }
+{ "l_orderkey": 5381, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18158.72, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly final requests haggle qui" }
+{ "l_orderkey": 5638, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12817.92, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "n, even requests. furiously ironic not" }
+{ "l_orderkey": 5700, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25635.84, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ix carefully " }
+{ "l_orderkey": 129, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e carefully blithely bold dolp" }
+{ "l_orderkey": 358, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1993-11-04", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng the ironic theo" }
+{ "l_orderkey": 709, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10691.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-06-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts cajole boldly " }
+{ "l_orderkey": 736, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 34213.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "iously final accoun" }
+{ "l_orderkey": 865, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36351.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-07-28", "l_receiptdate": "1993-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "furiously fluffily unusual account" }
+{ "l_orderkey": 928, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31005.64, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-17", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of the s" }
+{ "l_orderkey": 1057, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11760.76, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-31", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly final theodolites. furi" }
+{ "l_orderkey": 1153, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53458.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-07-13", "l_receiptdate": "1996-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic asymptotes nag slyly. " }
+{ "l_orderkey": 1220, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26729.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular orbi" }
+{ "l_orderkey": 1376, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23521.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "inst the final, pending " }
+{ "l_orderkey": 1506, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4276.64, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-12-06", "l_receiptdate": "1993-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "posits. furiou" }
+{ "l_orderkey": 1636, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ely express reque" }
+{ "l_orderkey": 1666, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 43835.56, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1996-01-04", "l_receiptdate": "1995-12-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly regular excuses; regular ac" }
+{ "l_orderkey": 1732, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26729.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-15", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nag slyly. even, special de" }
+{ "l_orderkey": 1894, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ily furiously bold packages. flu" }
+{ "l_orderkey": 1959, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49181.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously ex" }
+{ "l_orderkey": 2022, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40628.08, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-04-20", "l_receiptdate": "1992-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " against the express accounts wake ca" }
+{ "l_orderkey": 2146, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29936.48, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-17", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "r accounts sleep furio" }
+{ "l_orderkey": 2279, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9622.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ns cajole after the final platelets. s" }
+{ "l_orderkey": 2309, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits alongside of the final re" }
+{ "l_orderkey": 2369, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50250.52, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " to the regular dep" }
+{ "l_orderkey": 2560, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43835.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-11-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " after the accounts. regular foxes are be" }
+{ "l_orderkey": 3301, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nusual, final excuses after the entici" }
+{ "l_orderkey": 3648, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14968.24, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly pending excuses. carefully i" }
+{ "l_orderkey": 3713, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al pinto beans affix after the slyly " }
+{ "l_orderkey": 3715, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17106.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "usly regular pearls haggle final packages" }
+{ "l_orderkey": 3778, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29936.48, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y silent orbits print carefully against " }
+{ "l_orderkey": 3810, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19244.88, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-28", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. furiously careful deposi" }
+{ "l_orderkey": 3872, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19244.88, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. regular, brave accounts sleep blith" }
+{ "l_orderkey": 4578, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 44904.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-11-06", "l_receiptdate": "1993-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are caref" }
+{ "l_orderkey": 5095, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "carefully unusual plat" }
+{ "l_orderkey": 5317, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 32074.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-10-31", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "cross the attainments. slyly " }
+{ "l_orderkey": 5573, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 45973.88, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " furiously pending packages against " }
+{ "l_orderkey": 5635, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 40628.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-10-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly pendin" }
+{ "l_orderkey": 5953, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24590.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he silent ideas. silent foxes po" }
+{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
+{ "l_orderkey": 102, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "eposits cajole across" }
+{ "l_orderkey": 448, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8561.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-02", "l_commitdate": "1995-10-16", "l_receiptdate": "1995-11-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts wake blithely. furiously pending" }
+{ "l_orderkey": 545, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4280.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-23", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ", ironic grouches cajole over" }
+{ "l_orderkey": 645, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50297.99, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hely regular instructions alon" }
+{ "l_orderkey": 738, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nic, final excuses promise quickly regula" }
+{ "l_orderkey": 1221, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12842.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-07", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly ironic " }
+{ "l_orderkey": 1408, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ep along the fina" }
+{ "l_orderkey": 1444, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44947.14, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1995-03-03", "l_receiptdate": "1994-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold packages boost regular ideas. spe" }
+{ "l_orderkey": 1634, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11771.87, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "final requests " }
+{ "l_orderkey": 1954, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 31034.93, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-25", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "use thinly furiously regular asy" }
+{ "l_orderkey": 2309, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14982.38, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1995-10-22", "l_receiptdate": "1996-01-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "asymptotes. furiously pending acco" }
+{ "l_orderkey": 2342, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 53508.5, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-18", "l_receiptdate": "1996-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cial asymptotes pr" }
+{ "l_orderkey": 2406, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19263.06, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-02-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "azzle furiously careful" }
+{ "l_orderkey": 2596, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ily special re" }
+{ "l_orderkey": 2752, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22473.57, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " along the quickly " }
+{ "l_orderkey": 2854, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-09-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "age carefully" }
+{ "l_orderkey": 2944, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2140.34, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1997-12-01", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "luffily expr" }
+{ "l_orderkey": 3107, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-11-11", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "atelets must ha" }
+{ "l_orderkey": 3747, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35315.61, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular p" }
+{ "l_orderkey": 3874, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22473.57, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests cajole fluff" }
+{ "l_orderkey": 3905, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-07", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ow furiously. deposits wake ironic " }
+{ "l_orderkey": 4192, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28894.59, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully even escapades. care" }
+{ "l_orderkey": 4513, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31034.93, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole. regular packages boost. s" }
+{ "l_orderkey": 4516, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "even pinto beans wake qui" }
+{ "l_orderkey": 4960, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 44947.14, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-04-11", "l_receiptdate": "1995-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s requests cajole. " }
+{ "l_orderkey": 5793, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "snooze quick" }
+{ "l_orderkey": 422, "l_partkey": 171, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10711.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-08-04", "l_receiptdate": "1997-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he furiously ironic theodolite" }
+{ "l_orderkey": 455, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11782.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "g deposits against the slyly idle foxes u" }
+{ "l_orderkey": 545, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19281.06, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-01-17", "l_receiptdate": "1996-02-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "al, final packages affix. even a" }
+{ "l_orderkey": 1378, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-16", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "notornis. b" }
+{ "l_orderkey": 1639, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43917.97, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-19", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "structions w" }
+{ "l_orderkey": 2276, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28921.59, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-30", "l_commitdate": "1996-06-10", "l_receiptdate": "1996-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "the carefully unusual accoun" }
+{ "l_orderkey": 2310, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6427.02, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e slyly about the quickly ironic theodo" }
+{ "l_orderkey": 3361, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35348.61, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uriously ironic accounts. ironic, ir" }
+{ "l_orderkey": 3392, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42846.8, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-18", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ress instructions affix carefully. fur" }
+{ "l_orderkey": 3430, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 16067.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "cajole around the accounts. qui" }
+{ "l_orderkey": 3811, "l_partkey": 171, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53558.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-28", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ts are slyly fluffy ideas. furiou" }
+{ "l_orderkey": 3846, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32135.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits according to the fur" }
+{ "l_orderkey": 4066, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 46060.31, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-16", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "r instructions. slyly special " }
+{ "l_orderkey": 4134, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "kly above the quickly regular " }
+{ "l_orderkey": 4546, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16067.55, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-31", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ught to cajole furiously. qu" }
+{ "l_orderkey": 4675, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6427.02, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-06", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual ideas thrash bl" }
+{ "l_orderkey": 5317, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19281.06, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-02", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "g to the blithely p" }
+{ "l_orderkey": 5444, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22494.57, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "aves serve sly" }
+{ "l_orderkey": 167, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28948.59, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "eans affix furiously-- packages" }
+{ "l_orderkey": 225, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4288.68, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng the ironic packages. asymptotes among " }
+{ "l_orderkey": 1156, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45031.14, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-18", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-02-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. quickly bold pains are" }
+{ "l_orderkey": 1600, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21443.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "pths sleep blithely about the" }
+{ "l_orderkey": 1634, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 47175.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "requests affix slyly. quickly even pack" }
+{ "l_orderkey": 1889, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13938.21, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to the regular accounts. carefully express" }
+{ "l_orderkey": 2247, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12866.04, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final accounts. requests across the furiou" }
+{ "l_orderkey": 2817, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37525.95, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "gular foxes" }
+{ "l_orderkey": 3138, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40742.46, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely fluffily un" }
+{ "l_orderkey": 3751, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39670.29, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-30", "l_commitdate": "1994-05-30", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly express courts " }
+{ "l_orderkey": 4321, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10721.7, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "wake carefully alongside of " }
+{ "l_orderkey": 4448, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12866.04, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-21", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits about the ironic, bu" }
+{ "l_orderkey": 4544, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20371.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "regular ideas are furiously about" }
+{ "l_orderkey": 5255, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32165.1, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " to the silent requests cajole b" }
+{ "l_orderkey": 5572, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28948.59, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " accounts. carefully final accoun" }
+{ "l_orderkey": 5637, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37525.95, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-08-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s sleep blithely alongside of the ironic" }
+{ "l_orderkey": 5671, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13938.21, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-02", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "bold theodolites about" }
+{ "l_orderkey": 5988, "l_partkey": 172, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43958.97, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-02-06", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the pending, express reque" }
+{ "l_orderkey": 448, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49365.82, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-31", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " to the fluffily ironic packages." }
+{ "l_orderkey": 672, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43999.97, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-20", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " dependencies in" }
+{ "l_orderkey": 995, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16097.55, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-30", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses. fluffily fina" }
+{ "l_orderkey": 996, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 46146.31, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " the blithely ironic foxes. slyly silent d" }
+{ "l_orderkey": 1312, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19317.06, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-07-08", "l_receiptdate": "1994-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly ironic" }
+{ "l_orderkey": 1408, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7512.19, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-14", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fully final instructions. theodolites ca" }
+{ "l_orderkey": 1540, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40780.46, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " final grouches bo" }
+{ "l_orderkey": 1664, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ess multip" }
+{ "l_orderkey": 2466, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20390.23, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts cajole a" }
+{ "l_orderkey": 2534, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 18243.89, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-08-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "riously regular " }
+{ "l_orderkey": 2561, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "larly pending t" }
+{ "l_orderkey": 2630, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 48292.65, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-01-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "edly express ideas. carefully final " }
+{ "l_orderkey": 2853, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42926.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly. pearls cajole. final accounts ca" }
+{ "l_orderkey": 2945, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10731.7, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-03-10", "l_receiptdate": "1996-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thely. final courts could hang qu" }
+{ "l_orderkey": 3238, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27902.42, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g accounts sleep furiously ironic attai" }
+{ "l_orderkey": 3398, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1073.17, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " blithely final deposits." }
+{ "l_orderkey": 3495, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25756.08, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-10", "l_receiptdate": "1996-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ic, final pains along the even request" }
+{ "l_orderkey": 3654, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 48292.65, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-15", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sly ironic notornis nag slyly" }
+{ "l_orderkey": 3749, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11804.87, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular requests along the " }
+{ "l_orderkey": 3812, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35414.61, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal excuses d" }
+{ "l_orderkey": 3814, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 15024.38, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-17", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sits along the final, ironic deposit" }
+{ "l_orderkey": 3840, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7512.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-17", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". furiously final gifts sleep carefully pin" }
+{ "l_orderkey": 4545, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40780.46, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nts serve according to th" }
+{ "l_orderkey": 4583, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 28975.59, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-12-24", "l_receiptdate": "1995-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " detect silent requests. furiously speci" }
+{ "l_orderkey": 4608, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-10-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s cajole. slyly " }
+{ "l_orderkey": 4738, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits are slyly! carefu" }
+{ "l_orderkey": 4774, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "regular dolphins above the furi" }
+{ "l_orderkey": 4869, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 45073.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-12-10", "l_receiptdate": "1994-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly even instructions. " }
+{ "l_orderkey": 4964, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 30048.76, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-15", "l_receiptdate": "1997-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "among the carefully regula" }
+{ "l_orderkey": 5153, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34341.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-09-25", "l_receiptdate": "1996-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. ironi" }
+{ "l_orderkey": 5284, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "unts detect furiously even d" }
+{ "l_orderkey": 5377, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 28975.59, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "press theodolites. e" }
+{ "l_orderkey": 5605, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3219.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. accounts boost. t" }
+{ "l_orderkey": 5664, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9658.53, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-08-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ironic deposits haggle furiously. re" }
+{ "l_orderkey": 66, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-01", "l_receiptdate": "1994-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular de" }
+{ "l_orderkey": 67, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5370.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-20", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y unusual packages thrash pinto " }
+{ "l_orderkey": 261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30076.76, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-24", "l_commitdate": "1993-08-20", "l_receiptdate": "1993-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ironic packages nag slyly. carefully fin" }
+{ "l_orderkey": 289, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26854.25, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "out the quickly bold theodol" }
+{ "l_orderkey": 580, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ose alongside of the sl" }
+{ "l_orderkey": 775, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22557.57, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " quickly sile" }
+{ "l_orderkey": 871, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4296.68, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-01-20", "l_receiptdate": "1996-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l, regular dependencies w" }
+{ "l_orderkey": 1287, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-09-12", "l_receiptdate": "1994-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s wake unusual grou" }
+{ "l_orderkey": 1606, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully sil" }
+{ "l_orderkey": 1767, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25780.08, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-16", "l_commitdate": "1995-04-29", "l_receiptdate": "1995-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "luffy theodolites need to detect furi" }
+{ "l_orderkey": 1857, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular, regular inst" }
+{ "l_orderkey": 1991, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6445.02, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hes nag slyly" }
+{ "l_orderkey": 2054, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15038.38, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uickly final" }
+{ "l_orderkey": 2213, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 41892.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "the blithely " }
+{ "l_orderkey": 2305, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3222.51, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-24", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-03-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "kages haggle quickly across the blithely " }
+{ "l_orderkey": 2535, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26854.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ions believe ab" }
+{ "l_orderkey": 2820, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24705.91, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-10", "l_commitdate": "1994-08-08", "l_receiptdate": "1994-07-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " was furiously. deposits among the ironic" }
+{ "l_orderkey": 2950, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 29002.59, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-10-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "are alongside of the carefully silent " }
+{ "l_orderkey": 3010, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23631.74, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-06", "l_commitdate": "1996-04-06", "l_receiptdate": "1996-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final deposit" }
+{ "l_orderkey": 3360, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. carefully even deposits wake acros" }
+{ "l_orderkey": 3687, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10741.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-03-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ing pinto beans" }
+{ "l_orderkey": 3719, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2148.34, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-04-25", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ccounts boost carefu" }
+{ "l_orderkey": 4097, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45115.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "carefully silent foxes are against the " }
+{ "l_orderkey": 4261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 38670.12, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly pendi" }
+{ "l_orderkey": 4359, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s affix sly" }
+{ "l_orderkey": 4450, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47263.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-12", "l_commitdate": "1997-10-13", "l_receiptdate": "1997-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " the slyly eve" }
+{ "l_orderkey": 4613, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "against the quickly r" }
+{ "l_orderkey": 5063, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46189.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "latelets might nod blithely regular requ" }
+{ "l_orderkey": 5606, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50485.99, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "carefully final foxes. pending, final" }
+{ "l_orderkey": 5765, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51560.16, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "theodolites integrate furiously" }
+{ "l_orderkey": 5923, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 49411.82, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "nto beans cajole blithe" }
+{ "l_orderkey": 227, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25804.08, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uses across the blithe dependencies cajol" }
+{ "l_orderkey": 416, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26879.25, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-12-03", "l_receiptdate": "1993-10-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ses boost after the bold requests." }
+{ "l_orderkey": 738, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32255.1, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-05-29", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ecial instructions haggle blithely regula" }
+{ "l_orderkey": 929, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 47307.48, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. excuses cajole. carefully regu" }
+{ "l_orderkey": 960, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34405.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-19", "l_commitdate": "1994-12-17", "l_receiptdate": "1995-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "around the blithe, even pl" }
+{ "l_orderkey": 993, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35480.61, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1995-11-28", "l_receiptdate": "1996-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the deposits affix agains" }
+{ "l_orderkey": 1127, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7526.19, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-05", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " idly pending pains " }
+{ "l_orderkey": 1280, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5375.85, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1993-02-11", "l_receiptdate": "1993-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "beans haggle. quickly bold instructions h" }
+{ "l_orderkey": 1601, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53758.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-23", "l_receiptdate": "1995-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas doubt" }
+{ "l_orderkey": 1604, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 16127.55, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-10", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ending realms along the special, p" }
+{ "l_orderkey": 2497, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 30104.76, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hely bold ideas. unusual instructions ac" }
+{ "l_orderkey": 2528, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37630.95, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ", even excuses. even," }
+{ "l_orderkey": 2593, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1075.17, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-23", "l_commitdate": "1993-10-25", "l_receiptdate": "1993-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " accounts wake slyly " }
+{ "l_orderkey": 2626, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2150.34, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-19", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uffy accounts haggle furiously above" }
+{ "l_orderkey": 2784, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43006.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas nag furiously never unusual " }
+{ "l_orderkey": 2885, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial deposits use bold" }
+{ "l_orderkey": 2915, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30104.76, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "yly special " }
+{ "l_orderkey": 3175, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47307.48, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "are carefully furiously ironic accounts. e" }
+{ "l_orderkey": 3200, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 26879.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-08", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-03-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly regular hockey players! pinto beans " }
+{ "l_orderkey": 3554, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34405.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". blithely ironic t" }
+{ "l_orderkey": 3557, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44081.97, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-30", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas breach c" }
+{ "l_orderkey": 3622, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-02-22", "l_receiptdate": "1996-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "are careful" }
+{ "l_orderkey": 4261, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3225.51, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly even deposits eat blithely alo" }
+{ "l_orderkey": 4294, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. blithely r" }
+{ "l_orderkey": 4485, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al accounts according to the slyly r" }
+{ "l_orderkey": 4579, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15052.38, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-01-08", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding theodolites. fluffil" }
+{ "l_orderkey": 4610, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 15052.38, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " ironic frays. dependencies detect blithel" }
+{ "l_orderkey": 5634, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23653.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "silently unusual foxes above the blithely" }
+{ "l_orderkey": 5762, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6451.02, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-05-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ironic dependencies doze carefu" }
+{ "l_orderkey": 5859, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53758.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly regular deposits use. ironic" }
+{ "l_orderkey": 5956, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-06", "l_commitdate": "1998-06-29", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lyly express theodol" }
+{ "l_orderkey": 38, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47351.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-11-17", "l_receiptdate": "1996-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s. blithely unusual theodolites am" }
+{ "l_orderkey": 68, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " requests are unusual, regular pinto " }
+{ "l_orderkey": 422, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " ideas. qu" }
+{ "l_orderkey": 453, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " furiously f" }
+{ "l_orderkey": 675, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 36589.78, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-10-07", "l_receiptdate": "1997-11-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final accounts unwind around the " }
+{ "l_orderkey": 769, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38742.12, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-10-15", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "es. furiously iro" }
+{ "l_orderkey": 800, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 27980.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "bove the pending requests." }
+{ "l_orderkey": 1253, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24751.91, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the slyly silent re" }
+{ "l_orderkey": 1826, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 15066.38, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously bold pinto beans are carefully ag" }
+{ "l_orderkey": 1958, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31208.93, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "d pinto beans" }
+{ "l_orderkey": 2020, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43046.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ently across the" }
+{ "l_orderkey": 2306, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 20447.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tainments nag furiously carefull" }
+{ "l_orderkey": 2789, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35513.61, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "deposits. ironic " }
+{ "l_orderkey": 2981, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8609.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-21", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ng to the f" }
+{ "l_orderkey": 3109, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46275.31, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding to the foxes. " }
+{ "l_orderkey": 3206, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1076.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-16", "l_receiptdate": "1996-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y unusual foxes cajole ab" }
+{ "l_orderkey": 3365, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52732.33, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1995-01-01", "l_receiptdate": "1995-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lyly unusual asymptotes. final" }
+{ "l_orderkey": 3590, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10761.7, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "t the quickly ironic" }
+{ "l_orderkey": 3813, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39818.29, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-13", "l_commitdate": "1998-09-19", "l_receiptdate": "1998-10-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ravely special packages haggle p" }
+{ "l_orderkey": 3841, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3228.51, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-24", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "foxes integrate " }
+{ "l_orderkey": 3907, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 51656.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nt asymptotes lose across th" }
+{ "l_orderkey": 4800, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-28", "l_receiptdate": "1992-02-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s sleep fluffily. furiou" }
+{ "l_orderkey": 4803, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50579.99, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly final excuses. slyly express requ" }
+{ "l_orderkey": 5184, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27980.42, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-12-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " packages are" }
+{ "l_orderkey": 5280, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1998-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "efully carefully pen" }
+{ "l_orderkey": 5602, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9685.53, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-09-14", "l_receiptdate": "1997-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar foxes; quickly ironic ac" }
+{ "l_orderkey": 5924, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-12-11", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions cajole carefully along the " }
+{ "l_orderkey": 5987, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21523.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-09-17", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ing excuses nag quickly always bold" }
+{ "l_orderkey": 133, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12926.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1998-01-15", "l_receiptdate": "1997-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts cajole fluffily quickly i" }
+{ "l_orderkey": 229, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3231.51, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "posits. furiously regular theodol" }
+{ "l_orderkey": 774, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s according to the deposits unwind ca" }
+{ "l_orderkey": 896, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar, pending packages. deposits are q" }
+{ "l_orderkey": 1093, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39855.29, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-06", "l_commitdate": "1997-10-08", "l_receiptdate": "1997-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le furiously across the carefully sp" }
+{ "l_orderkey": 1441, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5385.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he quickly enticing pac" }
+{ "l_orderkey": 1632, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50626.99, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts. blithely regular " }
+{ "l_orderkey": 1954, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 14003.21, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic instructions cajole" }
+{ "l_orderkey": 1956, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8617.36, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1992-11-24", "l_receiptdate": "1993-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully about the ironic, ironic de" }
+{ "l_orderkey": 2405, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24774.91, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "t wake blithely blithely regular idea" }
+{ "l_orderkey": 2754, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20466.23, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-05-06", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "latelets hag" }
+{ "l_orderkey": 2788, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17234.72, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " requests wake carefully. carefully si" }
+{ "l_orderkey": 2852, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6463.02, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-02", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts above the furiously un" }
+{ "l_orderkey": 3141, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34469.44, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-12-18", "l_receiptdate": "1995-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "oxes are quickly about t" }
+{ "l_orderkey": 3169, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49549.82, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites are fl" }
+{ "l_orderkey": 3296, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17234.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-12-27", "l_receiptdate": "1995-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "kages cajole carefully " }
+{ "l_orderkey": 3622, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9694.53, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1996-02-09", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "arefully. furiously regular ideas n" }
+{ "l_orderkey": 3680, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "packages. quickly fluff" }
+{ "l_orderkey": 3713, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20466.23, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-25", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tructions serve blithely around the furi" }
+{ "l_orderkey": 4387, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sleep slyly. blithely sl" }
+{ "l_orderkey": 4514, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29083.59, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". slyly sile" }
+{ "l_orderkey": 4548, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23697.74, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s. furiously ironic theodolites c" }
+{ "l_orderkey": 4577, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46318.31, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-24", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly accounts. carefully " }
+{ "l_orderkey": 4644, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4308.68, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests? pendi" }
+{ "l_orderkey": 4709, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26929.25, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-03-03", "l_receiptdate": "1996-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inst the ironic, regul" }
+{ "l_orderkey": 4871, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15080.38, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "inst the never ironic " }
+{ "l_orderkey": 5382, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15080.38, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " brave platelets. ev" }
+{ "l_orderkey": 5504, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7540.19, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-03-15", "l_receiptdate": "1993-05-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "packages detect furiously express reques" }
+{ "l_orderkey": 5923, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29083.59, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "arefully i" }
+{ "l_orderkey": 197, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8625.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-17", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y blithely even deposits. blithely fina" }
+{ "l_orderkey": 1059, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17250.72, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pinto " }
+{ "l_orderkey": 1120, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10781.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "dependencies. blithel" }
+{ "l_orderkey": 1123, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42048.63, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "rding to the furiously ironic requests: r" }
+{ "l_orderkey": 1187, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31266.93, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1993-02-09", "l_receiptdate": "1992-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "riously express ac" }
+{ "l_orderkey": 1284, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-11", "l_commitdate": "1996-03-04", "l_receiptdate": "1996-04-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar packages. special packages ac" }
+{ "l_orderkey": 1286, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-24", "l_commitdate": "1993-08-12", "l_receiptdate": "1993-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gged accoun" }
+{ "l_orderkey": 1382, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46361.31, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-02", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ress deposits. slyly ironic foxes are blit" }
+{ "l_orderkey": 1413, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19407.06, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-11", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-10-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly bold packages haggle quickly acr" }
+{ "l_orderkey": 1504, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9703.53, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-12", "l_receiptdate": "1992-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly regular courts." }
+{ "l_orderkey": 1538, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14016.21, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-26", "l_commitdate": "1995-07-30", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. packages sleep f" }
+{ "l_orderkey": 1575, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15094.38, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans breach among the furiously specia" }
+{ "l_orderkey": 1607, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51752.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-22", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ular forges. deposits a" }
+{ "l_orderkey": 1633, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 37735.95, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-09", "l_commitdate": "1995-12-02", "l_receiptdate": "1996-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly against the dolph" }
+{ "l_orderkey": 1671, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "luffily regular deposits" }
+{ "l_orderkey": 1825, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 35579.61, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-03-01", "l_receiptdate": "1993-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ne" }
+{ "l_orderkey": 1923, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-08", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "aggle carefully. furiously permanent" }
+{ "l_orderkey": 1926, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10781.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-06-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "usly bold accounts. express accounts" }
+{ "l_orderkey": 2181, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4312.68, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tes. slyly silent packages use along th" }
+{ "l_orderkey": 2209, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7547.19, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " quickly regular pack" }
+{ "l_orderkey": 2306, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37735.95, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-08-30", "l_receiptdate": "1995-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "raids along the furiously unusual asympto" }
+{ "l_orderkey": 2661, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33423.27, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e ironicall" }
+{ "l_orderkey": 3173, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-06", "l_commitdate": "1996-09-17", "l_receiptdate": "1996-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "express depo" }
+{ "l_orderkey": 3235, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-16", "l_commitdate": "1996-01-05", "l_receiptdate": "1996-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ldly ironic pinto beans" }
+{ "l_orderkey": 3393, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39892.29, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ss the slyly ironic pinto beans. ironic," }
+{ "l_orderkey": 3521, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 40970.46, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1992-12-10", "l_receiptdate": "1993-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ges hang q" }
+{ "l_orderkey": 3809, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 46361.31, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-06", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "yly ironic decoys; regular, iron" }
+{ "l_orderkey": 3909, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32345.1, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-10-14", "l_receiptdate": "1998-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly even deposits across the ironic notorni" }
+{ "l_orderkey": 3940, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35579.61, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-19", "l_receiptdate": "1996-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly ironic packages about the pending accou" }
+{ "l_orderkey": 4130, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47439.48, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-14", "l_commitdate": "1996-04-15", "l_receiptdate": "1996-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eaves haggle qui" }
+{ "l_orderkey": 4131, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34501.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-02", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " furiously regular asymptotes nod sly" }
+{ "l_orderkey": 4196, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49595.82, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-06-28", "l_receiptdate": "1998-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "according to t" }
+{ "l_orderkey": 4579, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 36657.78, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-26", "l_commitdate": "1996-02-22", "l_receiptdate": "1996-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "hely. carefully blithe dependen" }
+{ "l_orderkey": 4580, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1993-12-17", "l_receiptdate": "1994-02-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "o beans. f" }
+{ "l_orderkey": 4646, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28032.42, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-08-25", "l_receiptdate": "1996-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ix according to the slyly spe" }
+{ "l_orderkey": 4678, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43126.8, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-10-27", "l_receiptdate": "1998-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". final, unusual requests sleep thinl" }
+{ "l_orderkey": 5092, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11859.87, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-12-27", "l_receiptdate": "1995-12-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly against the slyly silen" }
+{ "l_orderkey": 5095, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45283.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ccounts. packages could have t" }
+{ "l_orderkey": 5443, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15094.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-27", "l_commitdate": "1996-11-11", "l_receiptdate": "1996-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s after the regular, regular deposits hag" }
+{ "l_orderkey": 5472, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 48517.65, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " idle packages. furi" }
+{ "l_orderkey": 5792, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36657.78, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-06-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "requests are against t" }
+{ "l_orderkey": 67, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31295.93, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ultipliers " }
+{ "l_orderkey": 384, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41008.46, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "totes cajole blithely against the even" }
+{ "l_orderkey": 898, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39929.29, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-17", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "packages sleep furiously" }
+{ "l_orderkey": 1188, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-29", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "althy packages. fluffily unusual ideas h" }
+{ "l_orderkey": 1287, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y quickly bold theodoli" }
+{ "l_orderkey": 1410, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19425.06, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-03", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gle furiously fluffily regular requests" }
+{ "l_orderkey": 1537, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53958.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "special packages haggle slyly at the silent" }
+{ "l_orderkey": 2182, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39929.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ges. blithely ironic" }
+{ "l_orderkey": 2343, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-10-26", "l_receiptdate": "1995-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "osits. unusual theodolites boost furio" }
+{ "l_orderkey": 3459, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33454.27, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-10-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y regular pain" }
+{ "l_orderkey": 4066, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52879.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-02-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ial braids. furiously final deposits sl" }
+{ "l_orderkey": 4193, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "uffily spe" }
+{ "l_orderkey": 4551, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28058.42, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "le. carefully dogged accounts use furiousl" }
+{ "l_orderkey": 4578, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16187.55, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular theodo" }
+{ "l_orderkey": 4642, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s are blithely. requests wake above the fur" }
+{ "l_orderkey": 4741, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43166.8, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " fluffily slow deposits. fluffily regu" }
+{ "l_orderkey": 4835, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19425.06, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-17", "l_commitdate": "1994-12-14", "l_receiptdate": "1995-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "eat furiously against the slyly " }
+{ "l_orderkey": 5315, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42087.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-12-29", "l_receiptdate": "1992-12-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly alongside of the ca" }
+{ "l_orderkey": 5922, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly regular deposits haggle quickly ins" }
+{ "l_orderkey": 70, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1080.18, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-03-05", "l_receiptdate": "1994-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "quickly. fluffily unusual theodolites c" }
+{ "l_orderkey": 326, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ily quickly bold ideas." }
+{ "l_orderkey": 512, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43207.2, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "quests are da" }
+{ "l_orderkey": 640, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23763.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "osits across the slyly regular theodo" }
+{ "l_orderkey": 708, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-28", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " requests. even, thin ideas" }
+{ "l_orderkey": 772, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10801.8, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "o the furiously final deposits. furi" }
+{ "l_orderkey": 899, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-21", "l_commitdate": "1998-05-28", "l_receiptdate": "1998-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ades impress carefully" }
+{ "l_orderkey": 966, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "efully final pinto beans. quickly " }
+{ "l_orderkey": 1253, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-03", "l_commitdate": "1993-04-16", "l_receiptdate": "1993-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lar foxes sleep furiously final, final pack" }
+{ "l_orderkey": 1605, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19443.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-13", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly regular foxes wake carefully. bol" }
+{ "l_orderkey": 1826, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6481.08, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-30", "l_commitdate": "1992-05-17", "l_receiptdate": "1992-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "kages. blithely silent" }
+{ "l_orderkey": 1923, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11881.98, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-09-04", "l_receiptdate": "1997-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ages wake slyly about the furiously regular" }
+{ "l_orderkey": 2084, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es against " }
+{ "l_orderkey": 2881, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17282.88, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-06-27", "l_receiptdate": "1992-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "usly bold " }
+{ "l_orderkey": 2949, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 41046.84, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se slyly requests. carefull" }
+{ "l_orderkey": 3431, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-26", "l_commitdate": "1993-10-13", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " sleep carefully ironically special" }
+{ "l_orderkey": 3652, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25924.32, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "the final p" }
+{ "l_orderkey": 3713, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quests cajole careful" }
+{ "l_orderkey": 3906, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16202.7, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "dependencies at the " }
+{ "l_orderkey": 4067, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19443.24, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e the slyly final packages d" }
+{ "l_orderkey": 4099, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 49688.28, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-29", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages nag requests." }
+{ "l_orderkey": 4515, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-15", "l_receiptdate": "1992-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ns. bold r" }
+{ "l_orderkey": 4642, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36726.12, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-05-11", "l_receiptdate": "1995-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "theodolites detect among the ironically sp" }
+{ "l_orderkey": 4868, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8641.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly special th" }
+{ "l_orderkey": 4964, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12962.16, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-09-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully silent instructions ca" }
+{ "l_orderkey": 5092, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-20", "l_commitdate": "1995-11-30", "l_receiptdate": "1996-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " deposits cajole furiously against the sly" }
+{ "l_orderkey": 5221, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17282.88, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-29", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ending request" }
+{ "l_orderkey": 5318, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28084.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al, express foxes. bold requests sleep alwa" }
+{ "l_orderkey": 5382, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6481.08, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-07", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes by the sl" }
+{ "l_orderkey": 770, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "osits. foxes cajole " }
+{ "l_orderkey": 807, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 51896.64, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "kly across the f" }
+{ "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" }
+{ "l_orderkey": 999, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 40003.66, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1994-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckly slyly unusual packages: packages hagg" }
+{ "l_orderkey": 1088, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-07-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully ironic packages. r" }
+{ "l_orderkey": 1095, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 40003.66, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-04", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-10-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". quickly even dolphins sle" }
+{ "l_orderkey": 1349, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-07", "l_commitdate": "1998-01-14", "l_receiptdate": "1998-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " express inst" }
+{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31354.22, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " haggle: closely even asymptot" }
+{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11892.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "furiously unusual packages play quickly " }
+{ "l_orderkey": 1890, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 17298.88, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ged pinto beans. regular, regular id" }
+{ "l_orderkey": 2209, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-09-02", "l_receiptdate": "1992-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly around the final packages. deposits ca" }
+{ "l_orderkey": 2566, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-12-22", "l_receiptdate": "1992-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ously ironic accounts" }
+{ "l_orderkey": 2656, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10811.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-28", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s nag regularly about the deposits. slyly" }
+{ "l_orderkey": 2821, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4324.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-15", "l_commitdate": "1993-10-02", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nding foxes." }
+{ "l_orderkey": 2854, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49734.28, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". furiously regular deposits across th" }
+{ "l_orderkey": 3013, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18380.06, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-05-02", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fully unusual account" }
+{ "l_orderkey": 3041, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "posits dazzle special p" }
+{ "l_orderkey": 3232, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3243.54, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-12-11", "l_receiptdate": "1992-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ily blithely ironic acco" }
+{ "l_orderkey": 3424, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-03", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-11-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "bits boost closely slyly p" }
+{ "l_orderkey": 3653, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18380.06, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-07-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gle slyly regular" }
+{ "l_orderkey": 3872, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30273.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "t after the carefully ironic excuses. f" }
+{ "l_orderkey": 4293, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits should boost along the " }
+{ "l_orderkey": 4417, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-23", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-10-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "press deposits promise stealthily amo" }
+{ "l_orderkey": 4675, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lent pinto beans" }
+{ "l_orderkey": 5031, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 33516.58, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ts across the even requests doze furiously" }
+{ "l_orderkey": 5540, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss dolphins haggle " }
+{ "l_orderkey": 384, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11903.98, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-02", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ash carefully" }
+{ "l_orderkey": 547, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3246.54, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-04", "l_commitdate": "1996-08-01", "l_receiptdate": "1996-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "pinto beans. ironi" }
+{ "l_orderkey": 737, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12986.16, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits after the slyly bold du" }
+{ "l_orderkey": 1057, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21643.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s wake bol" }
+{ "l_orderkey": 1122, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ptotes. quickl" }
+{ "l_orderkey": 1510, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8657.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely express" }
+{ "l_orderkey": 1954, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1082.18, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "te. furiously final deposits hag" }
+{ "l_orderkey": 2022, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17314.88, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ages wake slyly care" }
+{ "l_orderkey": 2084, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45451.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y fluffily even foxes. " }
+{ "l_orderkey": 2401, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42205.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-10-21", "l_receiptdate": "1997-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ould affix " }
+{ "l_orderkey": 3457, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-12", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully final excuses wake" }
+{ "l_orderkey": 3713, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-08-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "the regular dugouts wake furiously sil" }
+{ "l_orderkey": 3716, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27054.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-23", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fully unusual accounts. carefu" }
+{ "l_orderkey": 3719, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12986.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-05-04", "l_receiptdate": "1997-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "grate according to the " }
+{ "l_orderkey": 3810, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11903.98, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-11", "l_receiptdate": "1993-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the pending pinto beans. expr" }
+{ "l_orderkey": 3811, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 24890.14, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nstructions sleep quickly. slyly final " }
+{ "l_orderkey": 4035, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14068.34, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-07-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s. furiously even courts wake slyly" }
+{ "l_orderkey": 4706, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40040.66, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-20", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "kly final deposits c" }
+{ "l_orderkey": 5380, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15150.52, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-18", "l_commitdate": "1997-12-03", "l_receiptdate": "1998-01-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "final platelets." }
+{ "l_orderkey": 5505, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35711.94, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-11-11", "l_receiptdate": "1998-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely unusual excuses integrat" }
+{ "l_orderkey": 5634, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-10", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final ideas. deposits sleep. reg" }
+{ "l_orderkey": 5664, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9739.62, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-04", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly. express ideas agai" }
+{ "l_orderkey": 5824, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 45451.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts sleep. carefully regular accounts h" }
+{ "l_orderkey": 7, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12998.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss pinto beans wake against th" }
+{ "l_orderkey": 102, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bits. ironic accoun" }
+{ "l_orderkey": 260, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28162.68, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-02-06", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld theodolites boost fl" }
+{ "l_orderkey": 359, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24913.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1995-03-11", "l_receiptdate": "1995-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic courts snooze quickly furiously final fo" }
+{ "l_orderkey": 515, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11914.98, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-10-02", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly pending accounts haggle blithel" }
+{ "l_orderkey": 774, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53075.82, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1996-01-07", "l_receiptdate": "1995-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ess accounts are carefully " }
+{ "l_orderkey": 1570, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its. slyly regular sentiments" }
+{ "l_orderkey": 1571, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6499.08, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1993-02-13", "l_receiptdate": "1993-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " special, ironic depo" }
+{ "l_orderkey": 1602, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4332.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-31", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y. even excuses" }
+{ "l_orderkey": 1605, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ole carefully car" }
+{ "l_orderkey": 2150, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37911.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending dependen" }
+{ "l_orderkey": 2438, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 49826.28, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic requests cajole f" }
+{ "l_orderkey": 2533, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40077.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " haggle carefully " }
+{ "l_orderkey": 2721, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53075.82, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-14", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ounts poach carefu" }
+{ "l_orderkey": 3143, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sly unusual theodolites. slyly ev" }
+{ "l_orderkey": 3143, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23829.96, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "beans. fluf" }
+{ "l_orderkey": 3202, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32495.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-03-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven platelets. furiously final" }
+{ "l_orderkey": 3299, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly even request" }
+{ "l_orderkey": 3749, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15164.52, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "press instruc" }
+{ "l_orderkey": 3942, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6499.08, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ep ruthlessly carefully final accounts: s" }
+{ "l_orderkey": 4070, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2166.36, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ptotes affix" }
+{ "l_orderkey": 4710, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cross the blithely bold packages. silen" }
+{ "l_orderkey": 4834, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29245.86, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-10-27", "l_receiptdate": "1997-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "es nag blithe" }
+{ "l_orderkey": 4994, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31412.22, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-09-27", "l_receiptdate": "1996-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ptotes boost carefully" }
+{ "l_orderkey": 4998, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16247.7, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "heodolites sleep quickly." }
+{ "l_orderkey": 5122, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30329.04, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-03-29", "l_receiptdate": "1996-04-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "g the busily ironic accounts boos" }
+{ "l_orderkey": 5191, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7582.26, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-30", "l_receiptdate": "1995-03-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "eposits. express" }
+{ "l_orderkey": 5408, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8665.44, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-09-06", "l_receiptdate": "1992-11-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "thely regular hocke" }
+{ "l_orderkey": 5475, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10831.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-22", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ding to the deposits wake fina" }
+{ "l_orderkey": 5734, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31412.22, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions cajole final, express " }
+{ "l_orderkey": 5792, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34661.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are slyly against the ev" }
+{ "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
+{ "l_orderkey": 194, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-05-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular theodolites. regular, iron" }
+{ "l_orderkey": 322, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10841.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits grow slyly according to th" }
+{ "l_orderkey": 326, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-27", "l_receiptdate": "1995-08-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily furiously unusual accounts. " }
+{ "l_orderkey": 390, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial excuses. bold, pending packages" }
+{ "l_orderkey": 484, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-03-27", "l_receiptdate": "1997-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uctions wake. final, silent requests haggle" }
+{ "l_orderkey": 897, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28188.68, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-07-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "tions sleep according to the special" }
+{ "l_orderkey": 1024, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14094.34, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e slyly around the slyly special instructi" }
+{ "l_orderkey": 1092, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 52040.64, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "unusual accounts. fluffi" }
+{ "l_orderkey": 1286, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-11", "l_receiptdate": "1993-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironic pinto beans cajole furiously s" }
+{ "l_orderkey": 1731, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-18", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ngside of the even instruct" }
+{ "l_orderkey": 1763, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 2168.36, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1996-12-04", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even pinto beans snooze fluffi" }
+{ "l_orderkey": 1891, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19515.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-02-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " foxes above the carefu" }
+{ "l_orderkey": 1923, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the ideas: slyly pendin" }
+{ "l_orderkey": 1925, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-23", "l_receiptdate": "1992-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "usual pinto" }
+{ "l_orderkey": 2115, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46619.74, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully pending requests alongs" }
+{ "l_orderkey": 2116, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11925.98, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-09-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pinto beans. final, final sauternes play " }
+{ "l_orderkey": 2118, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4336.72, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-25", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "theodolites affix according " }
+{ "l_orderkey": 2273, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36862.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " furiously carefully bold de" }
+{ "l_orderkey": 2533, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21683.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-04-30", "l_receiptdate": "1997-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thless excuses are b" }
+{ "l_orderkey": 2695, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22767.78, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y regular pinto beans. evenly regular packa" }
+{ "l_orderkey": 2790, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20599.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-10-03", "l_receiptdate": "1994-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uffily even excuses. furiously thin" }
+{ "l_orderkey": 2886, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41198.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-01-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old requests along the fur" }
+{ "l_orderkey": 3105, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11925.98, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-07", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly bold depths caj" }
+{ "l_orderkey": 3303, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-04-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly regular pi" }
+{ "l_orderkey": 3394, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15178.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-07-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "arefully regular do" }
+{ "l_orderkey": 3397, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-03", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " regular packag" }
+{ "l_orderkey": 3525, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-08", "l_receiptdate": "1996-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " nag according " }
+{ "l_orderkey": 3655, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5420.9, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-17", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "riously bold pinto be" }
+{ "l_orderkey": 3810, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53124.82, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "cajole. fur" }
+{ "l_orderkey": 3904, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20599.42, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " excuses sleep slyly according to th" }
+{ "l_orderkey": 4064, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1997-01-05", "l_receiptdate": "1996-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "alongside of the f" }
+{ "l_orderkey": 4069, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ages. carefully regular " }
+{ "l_orderkey": 4583, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests haggle after the furiously " }
+{ "l_orderkey": 4705, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " above the furiously ev" }
+{ "l_orderkey": 4801, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uests hinder blithely against the instr" }
+{ "l_orderkey": 4992, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45535.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-19", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "foxes about the quickly final platele" }
+{ "l_orderkey": 5121, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even courts are blithely ironically " }
+{ "l_orderkey": 5380, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43367.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-27", "l_receiptdate": "1998-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ar asymptotes. blithely r" }
+{ "l_orderkey": 5472, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egrate carefully dependencies. " }
+{ "l_orderkey": 5474, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41198.84, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-07-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly beneath " }
+{ "l_orderkey": 5543, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously. slyly" }
+{ "l_orderkey": 580, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20618.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "mong the special packag" }
+{ "l_orderkey": 612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5425.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "structions. q" }
+{ "l_orderkey": 835, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30385.04, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-11", "l_receiptdate": "1996-01-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " fluffily furious pinto beans" }
+{ "l_orderkey": 1347, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24959.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ronic pinto beans. express reques" }
+{ "l_orderkey": 1666, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32555.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-30", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " breach evenly final accounts. r" }
+{ "l_orderkey": 1796, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8681.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "slyly bold accounts are furiously agains" }
+{ "l_orderkey": 2023, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9766.62, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts maintain blithely alongside of the" }
+{ "l_orderkey": 2790, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29299.86, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-09-27", "l_receiptdate": "1994-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ilent packages cajole. quickly ironic requ" }
+{ "l_orderkey": 3009, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41236.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal packages should haggle slyly. quickl" }
+{ "l_orderkey": 3173, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2170.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-18", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fluffily above t" }
+{ "l_orderkey": 3267, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35810.94, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-30", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es boost. " }
+{ "l_orderkey": 3296, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31470.22, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-11-26", "l_receiptdate": "1995-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss ideas are reg" }
+{ "l_orderkey": 3712, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14107.34, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-02-11", "l_receiptdate": "1992-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s around the furiously ironic account" }
+{ "l_orderkey": 4389, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4340.72, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " blithely even d" }
+{ "l_orderkey": 4577, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 46662.74, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "packages. " }
+{ "l_orderkey": 4609, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3255.54, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nstructions. furious instructions " }
+{ "l_orderkey": 4612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10851.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-11", "l_commitdate": "1993-11-19", "l_receiptdate": "1993-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual theodol" }
+{ "l_orderkey": 4643, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54259.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". ironic deposits cajo" }
+{ "l_orderkey": 4739, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33640.58, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely special pin" }
+{ "l_orderkey": 5574, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49918.28, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "arefully express requests wake furiousl" }
+{ "l_orderkey": 5634, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28214.68, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ptotes mold qu" }
+{ "l_orderkey": 129, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39102.48, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-25", "l_receiptdate": "1992-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "packages are care" }
+{ "l_orderkey": 230, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49964.28, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old packages ha" }
+{ "l_orderkey": 325, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " theodolites. " }
+{ "l_orderkey": 357, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39102.48, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-11-13", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d the carefully even requests. " }
+{ "l_orderkey": 481, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45619.56, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "mptotes are furiously among the iron" }
+{ "l_orderkey": 518, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 52136.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-03-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slyly final platelets; quickly even deposi" }
+{ "l_orderkey": 610, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18465.06, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "p quickly instead of the slyly pending foxe" }
+{ "l_orderkey": 613, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3258.54, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ccounts cajole. " }
+{ "l_orderkey": 710, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 13034.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-18", "l_commitdate": "1993-02-27", "l_receiptdate": "1993-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ions. slyly express theodolites al" }
+{ "l_orderkey": 870, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-09-11", "l_receiptdate": "1993-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly excuses. ironi" }
+{ "l_orderkey": 1573, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-24", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ymptotes could u" }
+{ "l_orderkey": 1670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44533.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al gifts. speci" }
+{ "l_orderkey": 2114, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28240.68, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-30", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-05-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar asymptotes sleep " }
+{ "l_orderkey": 2466, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17378.88, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans sl" }
+{ "l_orderkey": 2823, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11947.98, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-11-24", "l_receiptdate": "1995-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "bold requests nag blithely s" }
+{ "l_orderkey": 2947, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10861.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly special " }
+{ "l_orderkey": 3174, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6517.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously ironic" }
+{ "l_orderkey": 3206, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26068.32, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "encies sleep deposits--" }
+{ "l_orderkey": 3298, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29326.86, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-10", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar packages. regular deposit" }
+{ "l_orderkey": 3489, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20637.42, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-31", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-08-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "c deposits alongside of the pending, fu" }
+{ "l_orderkey": 3623, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7603.26, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-05", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "aves. slyly special packages cajole. fu" }
+{ "l_orderkey": 3653, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9775.62, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-03", "l_commitdate": "1994-05-19", "l_receiptdate": "1994-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slyly silent account" }
+{ "l_orderkey": 4069, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3258.54, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l packages. even, " }
+{ "l_orderkey": 4321, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24982.14, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-03", "l_commitdate": "1994-10-08", "l_receiptdate": "1994-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly even orbits slee" }
+{ "l_orderkey": 5092, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 45619.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1996-01-01", "l_receiptdate": "1995-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s use along t" }
+{ "l_orderkey": 5283, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1086.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-20", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits within the furio" }
+{ "l_orderkey": 5286, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 41274.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-26", "l_receiptdate": "1997-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "fluffily. special, ironic deposit" }
+{ "l_orderkey": 5444, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22809.78, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar packages haggle above th" }
+{ "l_orderkey": 5670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46705.74, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ests in place of the carefully sly depos" }
+{ "l_orderkey": 5891, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9775.62, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-02-27", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cajole carefully " }
+{ "l_orderkey": 39, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28266.68, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ckages across the slyly silent" }
+{ "l_orderkey": 293, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11958.98, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " affix carefully quickly special idea" }
+{ "l_orderkey": 614, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 52184.64, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-01-19", "l_receiptdate": "1993-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously special excuses haggle along the" }
+{ "l_orderkey": 741, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27179.5, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "accounts. blithely bold pa" }
+{ "l_orderkey": 1031, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 29353.86, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gular deposits cajole. blithely unus" }
+{ "l_orderkey": 1346, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32615.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-01", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-10-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " nag blithely. unusual, ru" }
+{ "l_orderkey": 1475, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-13", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". slyly bold re" }
+{ "l_orderkey": 1509, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 33702.58, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-14", "l_commitdate": "1993-08-21", "l_receiptdate": "1993-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ic deposits cajole carefully. quickly bold " }
+{ "l_orderkey": 1639, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-06", "l_receiptdate": "1995-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the regular packages. courts dou" }
+{ "l_orderkey": 2211, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19569.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-31", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-09-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "c grouches. slyly express pinto " }
+{ "l_orderkey": 2406, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27179.5, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-03", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "al, regular in" }
+{ "l_orderkey": 2849, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42400.02, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-07-18", "l_receiptdate": "1996-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s sleep furiously silently regul" }
+{ "l_orderkey": 2950, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-09-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ides the b" }
+{ "l_orderkey": 2951, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43487.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ial deposits wake fluffily about th" }
+{ "l_orderkey": 2980, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-12", "l_commitdate": "1996-10-27", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "elets. fluffily regular in" }
+{ "l_orderkey": 3334, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21743.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses nag furiously. instructions are ca" }
+{ "l_orderkey": 3558, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3261.54, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-28", "l_receiptdate": "1996-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l, final deposits haggle. fina" }
+{ "l_orderkey": 3650, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20656.42, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y even forges. fluffily furious accounts" }
+{ "l_orderkey": 3748, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5435.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-29", "l_commitdate": "1998-05-06", "l_receiptdate": "1998-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " regular accounts sleep quickly-- furious" }
+{ "l_orderkey": 3840, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-31", "l_commitdate": "1998-09-19", "l_receiptdate": "1998-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "o beans are. carefully final courts x" }
+{ "l_orderkey": 4391, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ep quickly after " }
+{ "l_orderkey": 4583, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46748.74, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-12-17", "l_receiptdate": "1994-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fully after the speci" }
+{ "l_orderkey": 4647, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2174.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " pinto beans believe furiously slyly silent" }
+{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9784.62, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-06-26", "l_receiptdate": "1992-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits serve slyly. unusual pint" }
+{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14133.34, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " wake. unusual platelets for the" }
+{ "l_orderkey": 4930, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38051.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-09", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lose slyly regular dependencies. fur" }
+{ "l_orderkey": 5600, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36964.12, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly above the stealthy ideas. permane" }
+{ "l_orderkey": 5827, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32615.4, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ounts may c" }
+{ "l_orderkey": 5920, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-01-03", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "across the carefully pending platelets" }
+{ "l_orderkey": 578, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 25028.14, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nstructions. ironic deposits" }
+{ "l_orderkey": 738, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4352.72, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ar packages. fluffily bo" }
+{ "l_orderkey": 739, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32645.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-07-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "above the even deposits. ironic requests" }
+{ "l_orderkey": 836, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6529.08, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1997-01-31", "l_receiptdate": "1996-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fully bold theodolites are daringly across" }
+{ "l_orderkey": 896, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 36998.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-05-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular, close requests cajo" }
+{ "l_orderkey": 962, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-09-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "efully bold packages run slyly caref" }
+{ "l_orderkey": 1251, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use quickly final packages. iron" }
+{ "l_orderkey": 1285, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42439.02, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-15", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uctions. car" }
+{ "l_orderkey": 1410, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular account" }
+{ "l_orderkey": 1859, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39174.48, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular requests. carefully unusual theo" }
+{ "l_orderkey": 2945, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36998.12, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-02-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "at the unusual theodolite" }
+{ "l_orderkey": 2979, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38086.3, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-06-11", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "old ideas beneath the blit" }
+{ "l_orderkey": 3079, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2176.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-10-25", "l_receiptdate": "1998-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y regular asymptotes doz" }
+{ "l_orderkey": 3169, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 13058.16, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-18", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "atelets. pac" }
+{ "l_orderkey": 3203, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-01", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e the blithely regular accounts boost f" }
+{ "l_orderkey": 3362, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50056.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-10-12", "l_receiptdate": "1995-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly bold packages. regular deposits cajol" }
+{ "l_orderkey": 3653, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44615.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic packages affix sly" }
+{ "l_orderkey": 3746, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3264.54, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-03", "l_commitdate": "1994-12-10", "l_receiptdate": "1994-11-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the silent ideas cajole carefully " }
+{ "l_orderkey": 3781, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42439.02, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts are carefully. ir" }
+{ "l_orderkey": 3936, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26116.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1997-01-01", "l_receiptdate": "1996-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns. accounts mold fl" }
+{ "l_orderkey": 4226, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29380.86, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-03", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly alongside of the slyly ironic pac" }
+{ "l_orderkey": 4320, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35909.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess asymptotes so" }
+{ "l_orderkey": 4547, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16322.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-08", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-12-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ets haggle. regular dinos affix fu" }
+{ "l_orderkey": 4742, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 33733.58, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-13", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ke slyly among the furiousl" }
+{ "l_orderkey": 4935, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 39174.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "requests across the quick" }
+{ "l_orderkey": 5155, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ole blithely slyly ironic " }
+{ "l_orderkey": 5381, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40262.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final deposits print carefully. unusua" }
+{ "l_orderkey": 5511, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al theodolites. blithely final de" }
+{ "l_orderkey": 5698, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nts. slyly quiet pinto beans nag carefu" }
+{ "l_orderkey": 5764, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4352.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-23", "l_receiptdate": "1993-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ily regular courts haggle" }
+{ "l_orderkey": 5766, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-16", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular the" }
+{ "l_orderkey": 134, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 28318.68, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " among the pending depos" }
+{ "l_orderkey": 512, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20694.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " sleep. requests alongside of the fluff" }
+{ "l_orderkey": 549, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the regular, furious excuses. carefu" }
+{ "l_orderkey": 583, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14159.34, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-23", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y sly theodolites. ironi" }
+{ "l_orderkey": 612, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 35942.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-01", "l_receiptdate": "1992-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "bove the blithely even ideas. careful" }
+{ "l_orderkey": 705, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50102.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss deposits. ironic packa" }
+{ "l_orderkey": 839, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51191.46, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-06", "l_receiptdate": "1995-11-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully final excuses about " }
+{ "l_orderkey": 1280, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6535.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits " }
+{ "l_orderkey": 1285, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4356.72, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-08-17", "l_receiptdate": "1992-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l packages sleep slyly quiet i" }
+{ "l_orderkey": 1286, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11980.98, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-07-30", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " slyly even packages. requ" }
+{ "l_orderkey": 1543, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45745.56, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-11", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "xpress instructions. regular acc" }
+{ "l_orderkey": 2049, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27229.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-31", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " excuses above the " }
+{ "l_orderkey": 2245, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15248.52, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-05-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nts. always unusual dep" }
+{ "l_orderkey": 2372, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11980.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " silent, pending de" }
+{ "l_orderkey": 2496, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39210.48, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ully ironic f" }
+{ "l_orderkey": 2565, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28318.68, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-09", "l_receiptdate": "1998-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " pinto beans about the slyly regula" }
+{ "l_orderkey": 2624, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 13070.16, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "er the quickly unu" }
+{ "l_orderkey": 2690, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3267.54, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-04", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". final reques" }
+{ "l_orderkey": 2725, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16337.7, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "? furiously regular a" }
+{ "l_orderkey": 2883, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51191.46, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-02-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ep carefully ironic" }
+{ "l_orderkey": 3270, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31586.22, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sly regular asymptotes. slyly dog" }
+{ "l_orderkey": 3427, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26140.32, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-07-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y bold, sly deposits. pendi" }
+{ "l_orderkey": 3430, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2178.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-07", "l_commitdate": "1995-01-28", "l_receiptdate": "1995-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sh furiously according to the evenly e" }
+{ "l_orderkey": 3459, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10891.8, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-10-17", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ". blithely ironic pinto beans above" }
+{ "l_orderkey": 3847, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7624.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-06", "l_receiptdate": "1993-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " about the blithely daring Tiresias. fl" }
+{ "l_orderkey": 4448, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14159.34, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-26", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "fluffily express accounts integrate furiou" }
+{ "l_orderkey": 4514, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending excuses. sl" }
+{ "l_orderkey": 4580, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 42478.02, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". fluffily final dolphins use furiously al" }
+{ "l_orderkey": 4674, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 38121.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le quickly after the express sent" }
+{ "l_orderkey": 4803, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22872.78, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-15", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " silent packages use. b" }
+{ "l_orderkey": 4805, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 49013.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-07-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously sly t" }
+{ "l_orderkey": 5413, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 5445.9, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-28", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-12-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tes are al" }
+{ "l_orderkey": 5542, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6535.08, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-14", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " foxes doubt. theodolites ca" }
+{ "l_orderkey": 131, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4360.76, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " are carefully slyly i" }
+{ "l_orderkey": 162, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es! final somas integrate" }
+{ "l_orderkey": 224, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44697.79, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "after the furiou" }
+{ "l_orderkey": 358, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-18", "l_commitdate": "1993-12-12", "l_receiptdate": "1993-10-31", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y final foxes sleep blithely sl" }
+{ "l_orderkey": 389, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "fts. courts eat blithely even dependenc" }
+{ "l_orderkey": 610, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-19", "l_receiptdate": "1995-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " ironic pinto beans haggle. blithe" }
+{ "l_orderkey": 643, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 51238.93, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-05", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y against " }
+{ "l_orderkey": 672, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9811.71, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-25", "l_commitdate": "1994-06-06", "l_receiptdate": "1994-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "haggle carefully carefully reg" }
+{ "l_orderkey": 704, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43607.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ggle quickly. r" }
+{ "l_orderkey": 871, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31615.51, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1996-01-27", "l_receiptdate": "1995-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ests are carefu" }
+{ "l_orderkey": 1059, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54509.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-15", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s impress furiously about" }
+{ "l_orderkey": 1185, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 13082.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-26", "l_receiptdate": "1992-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "instructions. daringly pend" }
+{ "l_orderkey": 1344, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31615.51, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ffily quiet foxes wake blithely. slyly " }
+{ "l_orderkey": 1510, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39246.84, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old deposits along the carefully" }
+{ "l_orderkey": 1607, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages haggle. regular requests boost s" }
+{ "l_orderkey": 1856, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15262.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-14", "l_commitdate": "1992-05-02", "l_receiptdate": "1992-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ans are even requests. deposits caj" }
+{ "l_orderkey": 2182, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3270.57, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y bold theodolites wi" }
+{ "l_orderkey": 2437, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28344.94, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-07-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly regular accounts." }
+{ "l_orderkey": 2885, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5450.95, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-13", "l_receiptdate": "1993-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s. slyly express th" }
+{ "l_orderkey": 3334, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nts sublate slyly express pack" }
+{ "l_orderkey": 3622, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50148.74, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits wake. blithe" }
+{ "l_orderkey": 3780, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43607.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-06", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gular deposits-- furiously regular " }
+{ "l_orderkey": 4034, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 52329.12, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " blithely regular requests play carefull" }
+{ "l_orderkey": 4327, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 42517.41, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "kages against the blit" }
+{ "l_orderkey": 4421, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-08-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly final pinto beans impress. bold " }
+{ "l_orderkey": 4481, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ackages haggle even, " }
+{ "l_orderkey": 4484, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 41427.22, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". even requests un" }
+{ "l_orderkey": 4487, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1090.19, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ithely final asym" }
+{ "l_orderkey": 4641, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49058.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " about the close " }
+{ "l_orderkey": 4679, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. bold, regular packa" }
+{ "l_orderkey": 4773, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11992.09, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-02", "l_commitdate": "1996-01-29", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "en accounts. slyly b" }
+{ "l_orderkey": 4807, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-04", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "efully even dolphins slee" }
+{ "l_orderkey": 4930, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 41427.22, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "bold requests sleep never" }
+{ "l_orderkey": 5154, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11992.09, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "luffily bold foxes. final" }
+{ "l_orderkey": 5413, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1998-01-03", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully special package" }
+{ "l_orderkey": 5446, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ously across the quic" }
+{ "l_orderkey": 5763, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 9811.71, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-09-01", "l_receiptdate": "1998-10-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " deposits. instru" }
+{ "l_orderkey": 5926, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25074.37, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-23", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ickly special packages among " }
+{ "l_orderkey": 5984, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 38156.65, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "le fluffily regula" }
+{ "l_orderkey": 163, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21823.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "tructions integrate b" }
+{ "l_orderkey": 358, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44738.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ely frets. furious deposits sleep " }
+{ "l_orderkey": 550, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33826.89, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "thely silent packages. unusual" }
+{ "l_orderkey": 993, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43647.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gle above the furiously " }
+{ "l_orderkey": 1031, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 48012.36, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "re slyly above the furio" }
+{ "l_orderkey": 1506, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30553.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " unwind carefully: theodolit" }
+{ "l_orderkey": 1574, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54559.5, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1997-02-14", "l_receiptdate": "1996-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "le regular, regular foxes. blithely e" }
+{ "l_orderkey": 1574, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6547.14, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-03-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e silent, final packages. speci" }
+{ "l_orderkey": 1632, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 51285.93, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g to the closely special no" }
+{ "l_orderkey": 1668, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9820.71, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-17", "l_commitdate": "1997-09-05", "l_receiptdate": "1997-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "wake furiously even instructions. sil" }
+{ "l_orderkey": 1792, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 49103.55, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1993-12-24", "l_receiptdate": "1994-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests are. ironic, regular asy" }
+{ "l_orderkey": 1859, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lar packages wake quickly exp" }
+{ "l_orderkey": 2176, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41465.22, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1993-01-14", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lithely ironic pinto beans. furious" }
+{ "l_orderkey": 2373, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-05-19", "l_receiptdate": "1994-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "auternes. blithely even pinto bea" }
+{ "l_orderkey": 2531, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39282.84, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-06-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic, bold packages. blithely e" }
+{ "l_orderkey": 3265, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30553.32, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n requests. quickly final dinos" }
+{ "l_orderkey": 3298, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully regular requ" }
+{ "l_orderkey": 3361, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 33826.89, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-09-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts. pending, regular accounts sleep fur" }
+{ "l_orderkey": 3363, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-28", "l_receiptdate": "1995-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he regular, brave deposits. f" }
+{ "l_orderkey": 3590, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts above the silent waters thrash f" }
+{ "l_orderkey": 3809, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es detect furiously sil" }
+{ "l_orderkey": 3909, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50194.74, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-08", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-10-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "the blithely unusual ideas" }
+{ "l_orderkey": 3971, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2182.38, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-15", "l_commitdate": "1996-08-12", "l_receiptdate": "1996-07-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "haggle abou" }
+{ "l_orderkey": 4421, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34918.08, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-09", "l_commitdate": "1997-06-03", "l_receiptdate": "1997-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar ideas eat among the furiousl" }
+{ "l_orderkey": 4485, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-02-07", "l_receiptdate": "1994-12-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "play according to the ironic, ironic" }
+{ "l_orderkey": 4519, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly slyly furious depth" }
+{ "l_orderkey": 4646, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26188.56, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic platelets lose carefully. blithely unu" }
+{ "l_orderkey": 4708, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19641.42, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-11", "l_commitdate": "1994-11-15", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "special, eve" }
+{ "l_orderkey": 4737, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s. fluffily regular " }
+{ "l_orderkey": 4868, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-04-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ys engage. th" }
+{ "l_orderkey": 5350, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 48012.36, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p above the ironic, pending dep" }
+{ "l_orderkey": 5443, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6547.14, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-17", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p fluffily foxe" }
+{ "l_orderkey": 5669, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7638.33, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-07-07", "l_receiptdate": "1996-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "yly regular requests lose blithely. careful" }
+{ "l_orderkey": 5699, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32735.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-13", "l_commitdate": "1992-10-01", "l_receiptdate": "1992-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the carefully final " }
+{ "l_orderkey": 5767, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45829.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithe deposi" }
+{ "l_orderkey": 5829, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " ironic excuses use fluf" }
+{ "l_orderkey": 5831, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2182.38, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-20", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "quickly silent req" }
+{ "l_orderkey": 5859, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29462.13, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across th" }
+{ "l_orderkey": 5861, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34918.08, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nt asymptotes. carefully express request" }
+{ "l_orderkey": 5952, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 12003.09, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-06-04", "l_receiptdate": "1997-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y nag blithely aga" }
+{ "l_orderkey": 262, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42595.41, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-02-18", "l_receiptdate": "1996-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual, regular requests" }
+{ "l_orderkey": 679, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9829.71, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1996-01-27", "l_receiptdate": "1996-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep slyly. entici" }
+{ "l_orderkey": 742, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17475.04, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites haggle carefully regul" }
+{ "l_orderkey": 742, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 53517.31, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " carefully bold foxes sle" }
+{ "l_orderkey": 743, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22935.99, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "d requests. packages afte" }
+{ "l_orderkey": 1153, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 5460.95, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "special excuses promi" }
+{ "l_orderkey": 1255, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 13106.28, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular, express accounts are " }
+{ "l_orderkey": 1283, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43687.6, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "riously. even, ironic instructions after" }
+{ "l_orderkey": 1445, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7645.33, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-25", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-05-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "structions: slyly regular re" }
+{ "l_orderkey": 1538, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29489.13, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-09-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ngly even packag" }
+{ "l_orderkey": 2339, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24028.18, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-01-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " furiously above " }
+{ "l_orderkey": 2500, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43687.6, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully unusual dolphins s" }
+{ "l_orderkey": 2593, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 12014.09, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-01", "l_commitdate": "1993-11-19", "l_receiptdate": "1993-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "express packages sleep bold re" }
+{ "l_orderkey": 3169, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 13106.28, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-05", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular d" }
+{ "l_orderkey": 3174, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-26", "l_receiptdate": "1996-02-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "leep quickly? slyly special platelets" }
+{ "l_orderkey": 3523, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39318.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. fluffily regu" }
+{ "l_orderkey": 3554, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44779.79, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-08-28", "l_receiptdate": "1995-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ent dependencies. sly" }
+{ "l_orderkey": 3685, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42595.41, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-19", "l_commitdate": "1992-04-06", "l_receiptdate": "1992-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic courts nag carefully after the " }
+{ "l_orderkey": 4291, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3276.57, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-21", "l_receiptdate": "1994-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tes sleep slyly above the quickly sl" }
+{ "l_orderkey": 4454, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49148.55, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests promise. packages print fur" }
+{ "l_orderkey": 4513, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, final excuses detect furi" }
+{ "l_orderkey": 4582, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18567.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-17", "l_commitdate": "1996-08-26", "l_receiptdate": "1996-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng packages. depo" }
+{ "l_orderkey": 5250, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29489.13, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l forges are. furiously unusual pin" }
+{ "l_orderkey": 5254, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34950.08, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts impress closely furi" }
+{ "l_orderkey": 5345, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50240.74, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-27", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "slyly special deposits. fin" }
+{ "l_orderkey": 5346, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y. fluffily bold accounts grow. furio" }
+{ "l_orderkey": 5381, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s after the f" }
+{ "l_orderkey": 5731, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-23", "l_receiptdate": "1997-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ngside of the quickly regular depos" }
+{ "l_orderkey": 5857, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15290.66, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily pendin" }
+{ "l_orderkey": 163, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5465.95, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " must belie" }
+{ "l_orderkey": 259, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1993-12-22", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " requests sleep" }
+{ "l_orderkey": 320, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14211.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-12-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he furiously regular pinto beans. car" }
+{ "l_orderkey": 710, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41541.22, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sts boost fluffily aft" }
+{ "l_orderkey": 898, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39354.84, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " after the carefully " }
+{ "l_orderkey": 1057, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31702.51, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-05", "l_commitdate": "1992-05-05", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es wake according to the q" }
+{ "l_orderkey": 1254, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-02-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely even deposits eat!" }
+{ "l_orderkey": 1383, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15304.66, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ole carefully silent requests. car" }
+{ "l_orderkey": 1440, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-11-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions boost. fluffily regul" }
+{ "l_orderkey": 1696, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22956.99, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-02-18", "l_receiptdate": "1998-05-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y players sleep along the final, pending " }
+{ "l_orderkey": 1923, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 53566.31, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-21", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully expre" }
+{ "l_orderkey": 2180, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-29", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ep furiously furiously final request" }
+{ "l_orderkey": 2244, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-12", "l_commitdate": "1993-03-09", "l_receiptdate": "1993-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "rate around the reques" }
+{ "l_orderkey": 2336, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21863.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-03-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the fi" }
+{ "l_orderkey": 2340, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22956.99, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-03-04", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " asymptotes. unusual theo" }
+{ "l_orderkey": 2403, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 29516.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "deposits sleep slyly special theodolit" }
+{ "l_orderkey": 3937, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "into beans. slyly silent orbits alongside o" }
+{ "l_orderkey": 4295, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "telets cajole bravely" }
+{ "l_orderkey": 4359, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34982.08, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-18", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites nag quietly caref" }
+{ "l_orderkey": 4837, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-06-18", "l_receiptdate": "1998-08-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "counts cajole slyly furiou" }
+{ "l_orderkey": 4964, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 24050.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-09-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "equests doubt quickly. caref" }
+{ "l_orderkey": 5058, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-09", "l_receiptdate": "1998-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " the special foxes " }
+{ "l_orderkey": 5285, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33888.89, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-04-05", "l_receiptdate": "1994-05-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ubt. quickly blithe " }
+{ "l_orderkey": 5477, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-07", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "regular, s" }
+{ "l_orderkey": 5666, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-13", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the even, final foxes. quickly iron" }
+{ "l_orderkey": 5795, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37168.46, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-07-30", "l_receiptdate": "1992-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al instructions must affix along the ironic" }
+{ "l_orderkey": 5954, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " always regular dolphins. furiously p" }
+{ "l_orderkey": 354, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26260.56, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent requests. regular, even accounts" }
+{ "l_orderkey": 963, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. slyly regular depe" }
+{ "l_orderkey": 1255, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50332.74, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-08-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons nag qui" }
+{ "l_orderkey": 1378, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31731.51, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ual packages are furiously blith" }
+{ "l_orderkey": 1536, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5470.95, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-08", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "requests sleep pe" }
+{ "l_orderkey": 1573, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 12036.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-23", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "nently pending" }
+{ "l_orderkey": 1954, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53615.31, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eans. final pinto beans sleep furiousl" }
+{ "l_orderkey": 2084, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 37202.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully ironic requests. fluffil" }
+{ "l_orderkey": 2214, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54709.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-06", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts. blith" }
+{ "l_orderkey": 2657, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 33919.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-27", "l_receiptdate": "1995-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re blithely " }
+{ "l_orderkey": 2917, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-03-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly about the regular accounts. carefully pe" }
+{ "l_orderkey": 3073, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17507.04, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "n requests. ironi" }
+{ "l_orderkey": 3174, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4376.76, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas sleep thi" }
+{ "l_orderkey": 3586, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2188.38, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he even, unusual decoy" }
+{ "l_orderkey": 3590, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 48144.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-15", "l_receiptdate": "1995-06-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s sleep after the regular platelets. blit" }
+{ "l_orderkey": 3716, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42673.41, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "even deposits." }
+{ "l_orderkey": 3842, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30637.32, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lly alongside of the" }
+{ "l_orderkey": 3942, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5470.95, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-27", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". fluffily pending deposits above the flu" }
+{ "l_orderkey": 4195, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20789.61, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-08-13", "l_receiptdate": "1993-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "telets sleep even requests. final, even i" }
+{ "l_orderkey": 4355, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15318.66, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-01-22", "l_receiptdate": "1997-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he furiously ironic accounts. quickly iro" }
+{ "l_orderkey": 4356, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38296.65, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-30", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "arefully ironic " }
+{ "l_orderkey": 4642, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 12036.09, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lithely express asympt" }
+{ "l_orderkey": 4803, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20789.61, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-03-25", "l_receiptdate": "1996-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sts. enticing, even" }
+{ "l_orderkey": 4931, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1094.19, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1994-12-19", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " furiously " }
+{ "l_orderkey": 4966, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6565.14, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1996-11-29", "l_receiptdate": "1996-12-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "d deposits are sly excuses. slyly iro" }
+{ "l_orderkey": 5062, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27354.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-15", "l_commitdate": "1992-11-17", "l_receiptdate": "1993-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uthless excuses ag" }
+{ "l_orderkey": 5188, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39390.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-03-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "packages? blithely s" }
+{ "l_orderkey": 5409, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29543.13, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites " }
+{ "l_orderkey": 103, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6571.14, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-11", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-10-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "cajole. carefully ex" }
+{ "l_orderkey": 230, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6571.14, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-25", "l_receiptdate": "1994-02-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sleep furiously about the p" }
+{ "l_orderkey": 612, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30665.32, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1992-12-11", "l_receiptdate": "1993-01-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular instructions affix bl" }
+{ "l_orderkey": 614, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22998.99, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-01-06", "l_receiptdate": "1993-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "arefully. slyly express packag" }
+{ "l_orderkey": 1156, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 45997.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1997-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "even requests boost ironic deposits. pe" }
+{ "l_orderkey": 1506, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 16427.85, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-10-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully fluffy packages-- caref" }
+{ "l_orderkey": 1667, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2190.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "pecial requests hag" }
+{ "l_orderkey": 1702, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50378.74, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-07-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y even foxes. carefully final dependencies " }
+{ "l_orderkey": 2018, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2190.38, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-07-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly ironic accounts against the slyly sly" }
+{ "l_orderkey": 2241, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41617.22, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-31", "l_receiptdate": "1993-08-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " silent, unusual d" }
+{ "l_orderkey": 2309, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22998.99, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-05", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-11-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "unts around the dolphins ar" }
+{ "l_orderkey": 2341, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8761.52, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns affix above the iron" }
+{ "l_orderkey": 2439, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36141.27, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "asymptotes wake packages-- furiously" }
+{ "l_orderkey": 2468, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 48188.36, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular, silent sheave" }
+{ "l_orderkey": 2690, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 13142.28, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nal, regular atta" }
+{ "l_orderkey": 2848, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19713.42, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "osits haggle. stealthily ironic packa" }
+{ "l_orderkey": 2883, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39426.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-02", "l_commitdate": "1995-03-14", "l_receiptdate": "1995-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests detect slyly special packages" }
+{ "l_orderkey": 3012, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-07", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly furious packages. silently unusua" }
+{ "l_orderkey": 3173, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38331.65, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " across the slyly even requests." }
+{ "l_orderkey": 3205, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly pending packages snooz" }
+{ "l_orderkey": 3239, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28474.94, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-21", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ngly pending platelets are fluff" }
+{ "l_orderkey": 3362, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 44902.79, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ake alongside of the " }
+{ "l_orderkey": 3397, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8761.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y final foxes" }
+{ "l_orderkey": 3648, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "egular instructions. slyly regular pinto" }
+{ "l_orderkey": 3744, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32855.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nts among " }
+{ "l_orderkey": 4135, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14237.47, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-16", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "efully special account" }
+{ "l_orderkey": 4355, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 35046.08, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-29", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y silent deposits. b" }
+{ "l_orderkey": 5252, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9856.71, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "x. slyly special depos" }
+{ "l_orderkey": 5731, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly unusual ideas above the " }
+{ "l_orderkey": 5857, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54759.5, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-12-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y regular d" }
+{ "l_orderkey": 71, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 37270.46, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s cajole. " }
+{ "l_orderkey": 259, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3288.57, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng slyly at the accounts." }
+{ "l_orderkey": 482, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tructions near the final, regular ideas de" }
+{ "l_orderkey": 549, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19731.42, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "furiously according to the ironic, regular " }
+{ "l_orderkey": 614, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32885.7, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-02-08", "l_receiptdate": "1993-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "tructions are f" }
+{ "l_orderkey": 768, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "out the ironic" }
+{ "l_orderkey": 1060, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously. furiously regular in" }
+{ "l_orderkey": 1154, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " even, special " }
+{ "l_orderkey": 1155, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly final pinto beans was." }
+{ "l_orderkey": 1539, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 23019.99, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts haggle. busy" }
+{ "l_orderkey": 1731, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25212.37, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rays? bold, express pac" }
+{ "l_orderkey": 1828, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12058.09, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " wake blithely " }
+{ "l_orderkey": 2115, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-07-29", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully bold accounts " }
+{ "l_orderkey": 2214, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "t the blithely" }
+{ "l_orderkey": 2306, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-27", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly " }
+{ "l_orderkey": 3106, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6577.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "symptotes. slyly bold platelets cajol" }
+{ "l_orderkey": 3717, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49328.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s the blithely unu" }
+{ "l_orderkey": 3845, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "kages. care" }
+{ "l_orderkey": 4000, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-03-14", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ve the even, fi" }
+{ "l_orderkey": 4034, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7673.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y even theodolites. slyly regular instru" }
+{ "l_orderkey": 4038, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43847.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t. slyly silent pinto beans amo" }
+{ "l_orderkey": 4128, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5480.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ake permanently " }
+{ "l_orderkey": 4230, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-11", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ar packages are " }
+{ "l_orderkey": 4263, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ideas for the carefully re" }
+{ "l_orderkey": 4289, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20827.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e carefully regular ideas. sl" }
+{ "l_orderkey": 4390, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-06-22", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld braids haggle atop the for" }
+{ "l_orderkey": 4583, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to beans haggle sly" }
+{ "l_orderkey": 4613, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 51520.93, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously special requests wak" }
+{ "l_orderkey": 4736, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28500.94, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1996-01-18", "l_receiptdate": "1996-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "efully speci" }
+{ "l_orderkey": 4803, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 46039.98, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " accounts affix quickly ar" }
+{ "l_orderkey": 4902, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r the furiously final fox" }
+{ "l_orderkey": 5185, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly blithe deposits. furi" }
+{ "l_orderkey": 5637, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-11", "l_receiptdate": "1996-09-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ickly ironic gifts. blithely even cour" }
+{ "l_orderkey": 5922, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9865.71, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "haggle slyly even packages. packages" }
+{ "l_orderkey": 5959, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14250.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar forges. deposits det" }
+{ "l_orderkey": 5986, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 27404.75, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions. slyly regular de" }
+{ "l_orderkey": 70, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14263.47, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly special packag" }
+{ "l_orderkey": 192, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. dependencies nag furiously alongside" }
+{ "l_orderkey": 258, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "silent frets nod daringly busy, bold" }
+{ "l_orderkey": 518, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-26", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " the bold, special deposits are carefully " }
+{ "l_orderkey": 548, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-11-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests haggle quickly eve" }
+{ "l_orderkey": 706, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ckey players. requests above the" }
+{ "l_orderkey": 961, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warhorses slee" }
+{ "l_orderkey": 1283, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 23040.99, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "fully regular " }
+{ "l_orderkey": 1378, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37304.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le furiously slyly final accounts. careful" }
+{ "l_orderkey": 1671, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50470.74, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". slyly bold instructions boost. furiousl" }
+{ "l_orderkey": 1892, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15360.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously about the furiously" }
+{ "l_orderkey": 2080, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic deposits haggle slyly carefully eve" }
+{ "l_orderkey": 2180, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uriously f" }
+{ "l_orderkey": 2789, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cording to the careful de" }
+{ "l_orderkey": 2790, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-12-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ments. slyly f" }
+{ "l_orderkey": 2820, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g multipliers. final c" }
+{ "l_orderkey": 2882, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31818.51, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. furiously ironic" }
+{ "l_orderkey": 3138, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal foxes affix slyly. fluffily regul" }
+{ "l_orderkey": 3296, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic ideas across the" }
+{ "l_orderkey": 3552, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19749.42, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s deposits against the blithely unusual pin" }
+{ "l_orderkey": 3587, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5485.95, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely regular decoys above the " }
+{ "l_orderkey": 3616, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly ironic accounts unwind b" }
+{ "l_orderkey": 3748, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans run carefully quic" }
+{ "l_orderkey": 3969, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28526.94, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily; braids detect." }
+{ "l_orderkey": 4064, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es boost. careful" }
+{ "l_orderkey": 4194, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 47179.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-06", "l_commitdate": "1994-12-09", "l_receiptdate": "1994-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "olites are after the exp" }
+{ "l_orderkey": 4773, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39498.84, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-03-03", "l_receiptdate": "1996-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " dependencies. quickly" }
+{ "l_orderkey": 5056, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodolites. ironic a" }
+{ "l_orderkey": 5185, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40596.03, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gainst the courts dazzle care" }
+{ "l_orderkey": 5313, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17555.04, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ckages wake carefully aga" }
+{ "l_orderkey": 5509, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3291.57, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " quickly fin" }
+{ "l_orderkey": 5536, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38401.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "c, final theo" }
+{ "l_orderkey": 32, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 35142.08, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely regular deposits. fluffily " }
+{ "l_orderkey": 295, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31847.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-09", "l_commitdate": "1994-12-08", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inst the carefully ironic pinto beans. blit" }
+{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
+{ "l_orderkey": 608, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " alongside of the regular tithes. sly" }
+{ "l_orderkey": 709, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ily regular deposits. sauternes was accor" }
+{ "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37338.46, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
+{ "l_orderkey": 805, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27454.75, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ide of the pending, sly requests. quickly f" }
+{ "l_orderkey": 865, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-24", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y even accounts. quickly bold decoys" }
+{ "l_orderkey": 896, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10981.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quickly even theodolites. carefully regu" }
+{ "l_orderkey": 1124, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1098.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-06", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " instructions cajole qu" }
+{ "l_orderkey": 1314, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "equests nag across the furious" }
+{ "l_orderkey": 1316, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36240.27, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "manently; blithely special deposits" }
+{ "l_orderkey": 1345, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-27", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sly. furiously final accounts are blithely " }
+{ "l_orderkey": 1891, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " accounts are furiou" }
+{ "l_orderkey": 2277, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". quickly unusual deposi" }
+{ "l_orderkey": 2407, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions wake stealt" }
+{ "l_orderkey": 2533, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ccounts. ironic, special accounts boo" }
+{ "l_orderkey": 3011, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nusual sentiments. carefully bold idea" }
+{ "l_orderkey": 3200, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly against the quiet packages. blith" }
+{ "l_orderkey": 3396, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 34043.89, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "l, express pinto beans. quic" }
+{ "l_orderkey": 3428, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-13", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly pending requests int" }
+{ "l_orderkey": 3494, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole blithely" }
+{ "l_orderkey": 3943, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " grow fluffily according to the " }
+{ "l_orderkey": 4002, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21963.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly even ins" }
+{ "l_orderkey": 4263, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "d accounts. daringly regular accounts hagg" }
+{ "l_orderkey": 4294, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 32945.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-09-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites. bold foxes affix ironic theodolite" }
+{ "l_orderkey": 4551, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 29651.13, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y along the slyly even " }
+{ "l_orderkey": 5061, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8785.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-10-31", "l_receiptdate": "1993-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "regular foxes. ir" }
+{ "l_orderkey": 5159, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39534.84, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-11-07", "l_receiptdate": "1997-02-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake." }
+{ "l_orderkey": 5186, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 48320.36, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "old, final accounts cajole sl" }
+{ "l_orderkey": 5761, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold accounts wake above the" }
+{ "l_orderkey": 135, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 23082.99, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-12", "l_receiptdate": "1996-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits believe. furiously regular p" }
+{ "l_orderkey": 225, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49463.55, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "the slyly even platelets use aro" }
+{ "l_orderkey": 231, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54959.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic packages haggle fluffily a" }
+{ "l_orderkey": 678, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 52761.12, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ithely. slyly express foxes" }
+{ "l_orderkey": 740, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31876.51, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-17", "l_receiptdate": "1995-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ntly bold pinto beans sleep quickl" }
+{ "l_orderkey": 804, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2198.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly silent " }
+{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
+{ "l_orderkey": 964, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42868.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "se furiously regular instructions. blith" }
+{ "l_orderkey": 999, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 45066.79, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-12-04", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "us depths. carefully ironic instruc" }
+{ "l_orderkey": 1024, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53860.31, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ts. asymptotes nag fur" }
+{ "l_orderkey": 1348, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fter the regu" }
+{ "l_orderkey": 1728, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kly sly theodolites." }
+{ "l_orderkey": 1792, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 38471.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-01-20", "l_receiptdate": "1994-02-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e against the quic" }
+{ "l_orderkey": 1954, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12091.09, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y carefully ironi" }
+{ "l_orderkey": 1985, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 32975.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-09-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uickly. instr" }
+{ "l_orderkey": 1988, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20884.61, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly about the slyly thin instructions. f" }
+{ "l_orderkey": 2115, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14289.47, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-07", "l_commitdate": "1998-08-06", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans. even accounts abou" }
+{ "l_orderkey": 2535, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5495.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ", unusual reque" }
+{ "l_orderkey": 2752, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 41769.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es boost. slyly silent ideas" }
+{ "l_orderkey": 2818, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 24182.18, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egrate toward the carefully iron" }
+{ "l_orderkey": 2850, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4396.76, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al deposits cajole carefully quickly " }
+{ "l_orderkey": 3333, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39570.84, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes sleep neve" }
+{ "l_orderkey": 3495, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17587.04, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y bold dependencies; blithely idle sautern" }
+{ "l_orderkey": 3749, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-20", "l_receiptdate": "1995-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s. foxes sleep slyly unusual grouc" }
+{ "l_orderkey": 4002, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6595.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he slyly iro" }
+{ "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3297.57, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
+{ "l_orderkey": 4224, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29678.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-08-19", "l_receiptdate": "1997-09-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly special deposits sleep qui" }
+{ "l_orderkey": 4230, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 47265.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-03-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ses lose blithely slyly final e" }
+{ "l_orderkey": 4293, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51661.93, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely pending deposits af" }
+{ "l_orderkey": 5028, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16487.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-08-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular, bold pinto bea" }
+{ "l_orderkey": 5286, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1099.19, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly! furiously final pack" }
+{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
+{ "l_orderkey": 324, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-05-28", "l_receiptdate": "1992-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ross the slyly regular s" }
+{ "l_orderkey": 1121, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-16", "l_receiptdate": "1997-04-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "odolites. slyly even accounts" }
+{ "l_orderkey": 1254, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51709.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " platelets cajol" }
+{ "l_orderkey": 1411, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the" }
+{ "l_orderkey": 1447, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 45108.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rts boost s" }
+{ "l_orderkey": 1827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40707.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously ironic theodolites serve quickly af" }
+{ "l_orderkey": 2086, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely ironic acc" }
+{ "l_orderkey": 2304, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 46208.4, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests are blithely alongside of" }
+{ "l_orderkey": 2819, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eas after the carefully express pack" }
+{ "l_orderkey": 3008, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 34106.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold packages. quic" }
+{ "l_orderkey": 3169, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18703.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-21", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular packages. ironi" }
+{ "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42907.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
+{ "l_orderkey": 3363, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 4400.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ironic dependencie" }
+{ "l_orderkey": 3686, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41807.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent foxes! carefully ruthless cour" }
+{ "l_orderkey": 3878, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. regular instru" }
+{ "l_orderkey": 4064, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously f" }
+{ "l_orderkey": 4098, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50609.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly blithely silent deposits. fluff" }
+{ "l_orderkey": 4227, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2200.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ep. specia" }
+{ "l_orderkey": 4931, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s haggle al" }
+{ "l_orderkey": 5730, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s lose blithely. specia" }
+{ "l_orderkey": 5764, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ng to the fluffily qu" }
+{ "l_orderkey": 5827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-09-24", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully special packages wake thin" }
+{ "l_orderkey": 5895, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits wake blithely carefully fin" }
+{ "l_orderkey": 5952, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53909.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously regular" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm
new file mode 100644
index 0000000..beb05c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm
@@ -0,0 +1 @@
+[ [ { "v": { "id": 1, "f1": 40 } }, { "v": { "id": 2, "f1": 40 } }, { "v": { "id": 3, "f1": 40 } }, { "v": { "id": 7, "f1": 40 } }, { "v": { "id": 11, "f1": 40 } }, { "v": { "id": 15, "f1": 40 } }, { "v": { "id": 4, "f1": 41 } }, { "v": { "id": 8, "f1": 41 } }, { "v": { "id": 9, "f1": 41 } }, { "v": { "id": 12, "f1": 41 } }, { "v": { "id": 14, "f1": 41 } }, { "v": { "id": 18, "f1": 41 } }, { "v": { "id": 5, "f1": 42 } }, { "v": { "id": 6, "f1": 42 } }, { "v": { "id": 10, "f1": 42 } }, { "v": { "id": 13, "f1": 42 } }, { "v": { "id": 16, "f1": 42 } }, { "v": { "id": 17, "f1": 42 } } ] ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
index f2090a8..ae73d82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
@@ -1,105 +1,105 @@
 { "l_linenumber": 1, "l_l_orderkey": 1 }
 { "l_linenumber": 1, "l_l_orderkey": 2 }
+{ "l_linenumber": 1, "l_l_orderkey": 3 }
 { "l_linenumber": 1, "l_l_orderkey": 4 }
+{ "l_linenumber": 1, "l_l_orderkey": 5 }
+{ "l_linenumber": 1, "l_l_orderkey": 6 }
+{ "l_linenumber": 1, "l_l_orderkey": 7 }
+{ "l_linenumber": 1, "l_l_orderkey": 32 }
+{ "l_linenumber": 1, "l_l_orderkey": 33 }
+{ "l_linenumber": 1, "l_l_orderkey": 34 }
 { "l_linenumber": 1, "l_l_orderkey": 35 }
+{ "l_linenumber": 1, "l_l_orderkey": 36 }
 { "l_linenumber": 1, "l_l_orderkey": 37 }
+{ "l_linenumber": 1, "l_l_orderkey": 38 }
+{ "l_linenumber": 1, "l_l_orderkey": 39 }
 { "l_linenumber": 1, "l_l_orderkey": 64 }
 { "l_linenumber": 1, "l_l_orderkey": 65 }
 { "l_linenumber": 1, "l_l_orderkey": 66 }
 { "l_linenumber": 1, "l_l_orderkey": 67 }
 { "l_linenumber": 1, "l_l_orderkey": 68 }
 { "l_linenumber": 1, "l_l_orderkey": 69 }
-{ "l_linenumber": 1, "l_l_orderkey": 97 }
-{ "l_linenumber": 1, "l_l_orderkey": 98 }
-{ "l_linenumber": 1, "l_l_orderkey": 5 }
-{ "l_linenumber": 1, "l_l_orderkey": 6 }
-{ "l_linenumber": 1, "l_l_orderkey": 36 }
-{ "l_linenumber": 1, "l_l_orderkey": 38 }
-{ "l_linenumber": 1, "l_l_orderkey": 39 }
 { "l_linenumber": 1, "l_l_orderkey": 70 }
-{ "l_linenumber": 1, "l_l_orderkey": 3 }
-{ "l_linenumber": 1, "l_l_orderkey": 7 }
-{ "l_linenumber": 1, "l_l_orderkey": 32 }
-{ "l_linenumber": 1, "l_l_orderkey": 33 }
-{ "l_linenumber": 1, "l_l_orderkey": 34 }
 { "l_linenumber": 1, "l_l_orderkey": 71 }
 { "l_linenumber": 1, "l_l_orderkey": 96 }
+{ "l_linenumber": 1, "l_l_orderkey": 97 }
+{ "l_linenumber": 1, "l_l_orderkey": 98 }
 { "l_linenumber": 1, "l_l_orderkey": 99 }
-{ "l_linenumber": 2, "l_l_orderkey": 32 }
-{ "l_linenumber": 2, "l_l_orderkey": 34 }
-{ "l_linenumber": 2, "l_l_orderkey": 39 }
-{ "l_linenumber": 2, "l_l_orderkey": 67 }
 { "l_linenumber": 2, "l_l_orderkey": 1 }
 { "l_linenumber": 2, "l_l_orderkey": 3 }
-{ "l_linenumber": 2, "l_l_orderkey": 7 }
-{ "l_linenumber": 2, "l_l_orderkey": 33 }
-{ "l_linenumber": 2, "l_l_orderkey": 35 }
-{ "l_linenumber": 2, "l_l_orderkey": 66 }
 { "l_linenumber": 2, "l_l_orderkey": 5 }
+{ "l_linenumber": 2, "l_l_orderkey": 7 }
+{ "l_linenumber": 2, "l_l_orderkey": 32 }
+{ "l_linenumber": 2, "l_l_orderkey": 33 }
+{ "l_linenumber": 2, "l_l_orderkey": 34 }
+{ "l_linenumber": 2, "l_l_orderkey": 35 }
+{ "l_linenumber": 2, "l_l_orderkey": 37 }
+{ "l_linenumber": 2, "l_l_orderkey": 39 }
+{ "l_linenumber": 2, "l_l_orderkey": 65 }
+{ "l_linenumber": 2, "l_l_orderkey": 66 }
+{ "l_linenumber": 2, "l_l_orderkey": 67 }
+{ "l_linenumber": 2, "l_l_orderkey": 68 }
+{ "l_linenumber": 2, "l_l_orderkey": 69 }
 { "l_linenumber": 2, "l_l_orderkey": 70 }
 { "l_linenumber": 2, "l_l_orderkey": 71 }
 { "l_linenumber": 2, "l_l_orderkey": 96 }
-{ "l_linenumber": 2, "l_l_orderkey": 99 }
-{ "l_linenumber": 2, "l_l_orderkey": 37 }
-{ "l_linenumber": 2, "l_l_orderkey": 65 }
-{ "l_linenumber": 2, "l_l_orderkey": 68 }
-{ "l_linenumber": 2, "l_l_orderkey": 69 }
 { "l_linenumber": 2, "l_l_orderkey": 97 }
 { "l_linenumber": 2, "l_l_orderkey": 98 }
+{ "l_linenumber": 2, "l_l_orderkey": 99 }
 { "l_linenumber": 3, "l_l_orderkey": 1 }
 { "l_linenumber": 3, "l_l_orderkey": 3 }
+{ "l_linenumber": 3, "l_l_orderkey": 5 }
 { "l_linenumber": 3, "l_l_orderkey": 7 }
-{ "l_linenumber": 3, "l_l_orderkey": 33 }
-{ "l_linenumber": 3, "l_l_orderkey": 35 }
-{ "l_linenumber": 3, "l_l_orderkey": 70 }
-{ "l_linenumber": 3, "l_l_orderkey": 71 }
-{ "l_linenumber": 3, "l_l_orderkey": 69 }
-{ "l_linenumber": 3, "l_l_orderkey": 98 }
 { "l_linenumber": 3, "l_l_orderkey": 32 }
+{ "l_linenumber": 3, "l_l_orderkey": 33 }
 { "l_linenumber": 3, "l_l_orderkey": 34 }
+{ "l_linenumber": 3, "l_l_orderkey": 35 }
 { "l_linenumber": 3, "l_l_orderkey": 37 }
 { "l_linenumber": 3, "l_l_orderkey": 39 }
 { "l_linenumber": 3, "l_l_orderkey": 65 }
 { "l_linenumber": 3, "l_l_orderkey": 67 }
 { "l_linenumber": 3, "l_l_orderkey": 68 }
+{ "l_linenumber": 3, "l_l_orderkey": 69 }
+{ "l_linenumber": 3, "l_l_orderkey": 70 }
+{ "l_linenumber": 3, "l_l_orderkey": 71 }
 { "l_linenumber": 3, "l_l_orderkey": 97 }
-{ "l_linenumber": 3, "l_l_orderkey": 5 }
+{ "l_linenumber": 3, "l_l_orderkey": 98 }
 { "l_linenumber": 3, "l_l_orderkey": 99 }
 { "l_linenumber": 4, "l_l_orderkey": 1 }
-{ "l_linenumber": 4, "l_l_orderkey": 35 }
-{ "l_linenumber": 4, "l_l_orderkey": 67 }
-{ "l_linenumber": 4, "l_l_orderkey": 68 }
-{ "l_linenumber": 4, "l_l_orderkey": 71 }
 { "l_linenumber": 4, "l_l_orderkey": 3 }
 { "l_linenumber": 4, "l_l_orderkey": 7 }
 { "l_linenumber": 4, "l_l_orderkey": 32 }
 { "l_linenumber": 4, "l_l_orderkey": 33 }
+{ "l_linenumber": 4, "l_l_orderkey": 35 }
 { "l_linenumber": 4, "l_l_orderkey": 39 }
-{ "l_linenumber": 4, "l_l_orderkey": 70 }
+{ "l_linenumber": 4, "l_l_orderkey": 67 }
+{ "l_linenumber": 4, "l_l_orderkey": 68 }
 { "l_linenumber": 4, "l_l_orderkey": 69 }
+{ "l_linenumber": 4, "l_l_orderkey": 70 }
+{ "l_linenumber": 4, "l_l_orderkey": 71 }
 { "l_linenumber": 4, "l_l_orderkey": 98 }
 { "l_linenumber": 4, "l_l_orderkey": 99 }
 { "l_linenumber": 5, "l_l_orderkey": 1 }
+{ "l_linenumber": 5, "l_l_orderkey": 3 }
 { "l_linenumber": 5, "l_l_orderkey": 7 }
-{ "l_linenumber": 5, "l_l_orderkey": 35 }
-{ "l_linenumber": 5, "l_l_orderkey": 70 }
-{ "l_linenumber": 5, "l_l_orderkey": 71 }
-{ "l_linenumber": 5, "l_l_orderkey": 69 }
 { "l_linenumber": 5, "l_l_orderkey": 32 }
+{ "l_linenumber": 5, "l_l_orderkey": 35 }
 { "l_linenumber": 5, "l_l_orderkey": 39 }
 { "l_linenumber": 5, "l_l_orderkey": 67 }
 { "l_linenumber": 5, "l_l_orderkey": 68 }
-{ "l_linenumber": 5, "l_l_orderkey": 3 }
-{ "l_linenumber": 6, "l_l_orderkey": 3 }
+{ "l_linenumber": 5, "l_l_orderkey": 69 }
+{ "l_linenumber": 5, "l_l_orderkey": 70 }
+{ "l_linenumber": 5, "l_l_orderkey": 71 }
 { "l_linenumber": 6, "l_l_orderkey": 1 }
-{ "l_linenumber": 6, "l_l_orderkey": 35 }
-{ "l_linenumber": 6, "l_l_orderkey": 67 }
-{ "l_linenumber": 6, "l_l_orderkey": 68 }
-{ "l_linenumber": 6, "l_l_orderkey": 71 }
-{ "l_linenumber": 6, "l_l_orderkey": 69 }
+{ "l_linenumber": 6, "l_l_orderkey": 3 }
 { "l_linenumber": 6, "l_l_orderkey": 7 }
 { "l_linenumber": 6, "l_l_orderkey": 32 }
+{ "l_linenumber": 6, "l_l_orderkey": 35 }
 { "l_linenumber": 6, "l_l_orderkey": 39 }
+{ "l_linenumber": 6, "l_l_orderkey": 67 }
+{ "l_linenumber": 6, "l_l_orderkey": 68 }
+{ "l_linenumber": 6, "l_l_orderkey": 69 }
 { "l_linenumber": 6, "l_l_orderkey": 70 }
-{ "l_linenumber": 7, "l_l_orderkey": 68 }
+{ "l_linenumber": 6, "l_l_orderkey": 71 }
 { "l_linenumber": 7, "l_l_orderkey": 7 }
+{ "l_linenumber": 7, "l_l_orderkey": 68 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
index 83f8e7e..a04e7dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
@@ -1,551 +1,557 @@
+{ "l_orderkey": 324, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-05-28", "l_receiptdate": "1992-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ross the slyly regular s" }
 { "l_orderkey": 1121, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-16", "l_receiptdate": "1997-04-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "odolites. slyly even accounts" }
-{ "l_orderkey": 1447, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 45108.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rts boost s" }
-{ "l_orderkey": 2819, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eas after the carefully express pack" }
-{ "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42907.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
-{ "l_orderkey": 4064, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously f" }
+{ "l_orderkey": 1254, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51709.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " platelets cajol" }
 { "l_orderkey": 1411, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the" }
+{ "l_orderkey": 1447, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 45108.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rts boost s" }
+{ "l_orderkey": 1827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40707.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously ironic theodolites serve quickly af" }
 { "l_orderkey": 2086, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely ironic acc" }
+{ "l_orderkey": 2304, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 46208.4, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests are blithely alongside of" }
+{ "l_orderkey": 2819, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eas after the carefully express pack" }
+{ "l_orderkey": 3008, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 34106.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold packages. quic" }
+{ "l_orderkey": 3169, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18703.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-21", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular packages. ironi" }
+{ "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42907.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
 { "l_orderkey": 3363, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 4400.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ironic dependencie" }
 { "l_orderkey": 3686, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41807.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent foxes! carefully ruthless cour" }
 { "l_orderkey": 3878, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. regular instru" }
-{ "l_orderkey": 4931, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s haggle al" }
-{ "l_orderkey": 5827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-09-24", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully special packages wake thin" }
-{ "l_orderkey": 5895, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits wake blithely carefully fin" }
-{ "l_orderkey": 1254, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51709.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " platelets cajol" }
-{ "l_orderkey": 3008, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 34106.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold packages. quic" }
+{ "l_orderkey": 4064, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously f" }
 { "l_orderkey": 4098, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50609.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly blithely silent deposits. fluff" }
 { "l_orderkey": 4227, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2200.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ep. specia" }
-{ "l_orderkey": 324, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-05-28", "l_receiptdate": "1992-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ross the slyly regular s" }
-{ "l_orderkey": 1827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40707.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously ironic theodolites serve quickly af" }
-{ "l_orderkey": 2304, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 46208.4, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests are blithely alongside of" }
-{ "l_orderkey": 3169, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18703.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-21", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular packages. ironi" }
+{ "l_orderkey": 4931, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s haggle al" }
 { "l_orderkey": 5730, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s lose blithely. specia" }
 { "l_orderkey": 5764, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ng to the fluffily qu" }
+{ "l_orderkey": 5827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-09-24", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully special packages wake thin" }
+{ "l_orderkey": 5895, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits wake blithely carefully fin" }
 { "l_orderkey": 5952, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53909.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously regular" }
+{ "l_orderkey": 135, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 23082.99, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-12", "l_receiptdate": "1996-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits believe. furiously regular p" }
 { "l_orderkey": 225, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49463.55, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "the slyly even platelets use aro" }
+{ "l_orderkey": 231, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54959.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic packages haggle fluffily a" }
 { "l_orderkey": 678, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 52761.12, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ithely. slyly express foxes" }
 { "l_orderkey": 740, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31876.51, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-17", "l_receiptdate": "1995-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ntly bold pinto beans sleep quickl" }
+{ "l_orderkey": 804, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2198.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly silent " }
+{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
+{ "l_orderkey": 964, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42868.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "se furiously regular instructions. blith" }
 { "l_orderkey": 999, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 45066.79, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-12-04", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "us depths. carefully ironic instruc" }
 { "l_orderkey": 1024, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53860.31, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ts. asymptotes nag fur" }
-{ "l_orderkey": 2535, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5495.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ", unusual reque" }
-{ "l_orderkey": 3495, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17587.04, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y bold dependencies; blithely idle sautern" }
-{ "l_orderkey": 4230, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 47265.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-03-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ses lose blithely slyly final e" }
-{ "l_orderkey": 4293, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51661.93, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely pending deposits af" }
-{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
-{ "l_orderkey": 135, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 23082.99, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-12", "l_receiptdate": "1996-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits believe. furiously regular p" }
 { "l_orderkey": 1348, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fter the regu" }
+{ "l_orderkey": 1728, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kly sly theodolites." }
+{ "l_orderkey": 1792, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 38471.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-01-20", "l_receiptdate": "1994-02-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e against the quic" }
 { "l_orderkey": 1954, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12091.09, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y carefully ironi" }
 { "l_orderkey": 1985, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 32975.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-09-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uickly. instr" }
+{ "l_orderkey": 1988, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20884.61, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly about the slyly thin instructions. f" }
+{ "l_orderkey": 2115, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14289.47, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-07", "l_commitdate": "1998-08-06", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans. even accounts abou" }
+{ "l_orderkey": 2535, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5495.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ", unusual reque" }
+{ "l_orderkey": 2752, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 41769.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es boost. slyly silent ideas" }
+{ "l_orderkey": 2818, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 24182.18, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egrate toward the carefully iron" }
+{ "l_orderkey": 2850, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4396.76, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al deposits cajole carefully quickly " }
+{ "l_orderkey": 3333, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39570.84, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes sleep neve" }
+{ "l_orderkey": 3495, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17587.04, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y bold dependencies; blithely idle sautern" }
 { "l_orderkey": 3749, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-20", "l_receiptdate": "1995-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s. foxes sleep slyly unusual grouc" }
 { "l_orderkey": 4002, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6595.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he slyly iro" }
 { "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3297.57, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
-{ "l_orderkey": 804, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2198.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly silent " }
-{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
-{ "l_orderkey": 1988, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20884.61, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly about the slyly thin instructions. f" }
-{ "l_orderkey": 2752, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 41769.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es boost. slyly silent ideas" }
-{ "l_orderkey": 2818, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 24182.18, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egrate toward the carefully iron" }
-{ "l_orderkey": 3333, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39570.84, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes sleep neve" }
 { "l_orderkey": 4224, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29678.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-08-19", "l_receiptdate": "1997-09-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly special deposits sleep qui" }
+{ "l_orderkey": 4230, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 47265.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-03-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ses lose blithely slyly final e" }
+{ "l_orderkey": 4293, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51661.93, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely pending deposits af" }
 { "l_orderkey": 5028, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16487.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-08-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular, bold pinto bea" }
 { "l_orderkey": 5286, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1099.19, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly! furiously final pack" }
-{ "l_orderkey": 231, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54959.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic packages haggle fluffily a" }
-{ "l_orderkey": 964, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42868.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "se furiously regular instructions. blith" }
-{ "l_orderkey": 1728, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kly sly theodolites." }
-{ "l_orderkey": 1792, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 38471.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-01-20", "l_receiptdate": "1994-02-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e against the quic" }
-{ "l_orderkey": 2115, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14289.47, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-07", "l_commitdate": "1998-08-06", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans. even accounts abou" }
-{ "l_orderkey": 2850, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4396.76, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al deposits cajole carefully quickly " }
+{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
 { "l_orderkey": 32, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 35142.08, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely regular deposits. fluffily " }
-{ "l_orderkey": 709, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ily regular deposits. sauternes was accor" }
-{ "l_orderkey": 2533, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ccounts. ironic, special accounts boo" }
-{ "l_orderkey": 3428, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-13", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly pending requests int" }
-{ "l_orderkey": 4551, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 29651.13, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y along the slyly even " }
-{ "l_orderkey": 865, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-24", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y even accounts. quickly bold decoys" }
-{ "l_orderkey": 1124, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1098.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-06", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " instructions cajole qu" }
-{ "l_orderkey": 1345, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-27", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sly. furiously final accounts are blithely " }
-{ "l_orderkey": 2407, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions wake stealt" }
-{ "l_orderkey": 3011, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nusual sentiments. carefully bold idea" }
-{ "l_orderkey": 3200, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly against the quiet packages. blith" }
-{ "l_orderkey": 4263, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "d accounts. daringly regular accounts hagg" }
-{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
-{ "l_orderkey": 805, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27454.75, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ide of the pending, sly requests. quickly f" }
-{ "l_orderkey": 1891, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " accounts are furiou" }
-{ "l_orderkey": 2277, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". quickly unusual deposi" }
-{ "l_orderkey": 4002, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21963.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly even ins" }
-{ "l_orderkey": 5061, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8785.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-10-31", "l_receiptdate": "1993-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "regular foxes. ir" }
-{ "l_orderkey": 5761, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold accounts wake above the" }
 { "l_orderkey": 295, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31847.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-09", "l_commitdate": "1994-12-08", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inst the carefully ironic pinto beans. blit" }
+{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
 { "l_orderkey": 608, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " alongside of the regular tithes. sly" }
+{ "l_orderkey": 709, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ily regular deposits. sauternes was accor" }
 { "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37338.46, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
+{ "l_orderkey": 805, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27454.75, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ide of the pending, sly requests. quickly f" }
+{ "l_orderkey": 865, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-24", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y even accounts. quickly bold decoys" }
 { "l_orderkey": 896, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10981.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quickly even theodolites. carefully regu" }
+{ "l_orderkey": 1124, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1098.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-06", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " instructions cajole qu" }
 { "l_orderkey": 1314, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "equests nag across the furious" }
 { "l_orderkey": 1316, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36240.27, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "manently; blithely special deposits" }
+{ "l_orderkey": 1345, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-27", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sly. furiously final accounts are blithely " }
+{ "l_orderkey": 1891, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " accounts are furiou" }
+{ "l_orderkey": 2277, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". quickly unusual deposi" }
+{ "l_orderkey": 2407, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions wake stealt" }
+{ "l_orderkey": 2533, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ccounts. ironic, special accounts boo" }
+{ "l_orderkey": 3011, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nusual sentiments. carefully bold idea" }
+{ "l_orderkey": 3200, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly against the quiet packages. blith" }
 { "l_orderkey": 3396, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 34043.89, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "l, express pinto beans. quic" }
+{ "l_orderkey": 3428, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-13", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly pending requests int" }
 { "l_orderkey": 3494, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole blithely" }
 { "l_orderkey": 3943, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " grow fluffily according to the " }
+{ "l_orderkey": 4002, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21963.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly even ins" }
+{ "l_orderkey": 4263, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "d accounts. daringly regular accounts hagg" }
 { "l_orderkey": 4294, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 32945.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-09-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites. bold foxes affix ironic theodolite" }
+{ "l_orderkey": 4551, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 29651.13, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y along the slyly even " }
+{ "l_orderkey": 5061, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8785.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-10-31", "l_receiptdate": "1993-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "regular foxes. ir" }
 { "l_orderkey": 5159, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39534.84, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-11-07", "l_receiptdate": "1997-02-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake." }
 { "l_orderkey": 5186, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 48320.36, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "old, final accounts cajole sl" }
+{ "l_orderkey": 5761, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold accounts wake above the" }
+{ "l_orderkey": 70, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14263.47, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly special packag" }
 { "l_orderkey": 192, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. dependencies nag furiously alongside" }
+{ "l_orderkey": 258, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "silent frets nod daringly busy, bold" }
 { "l_orderkey": 518, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-26", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " the bold, special deposits are carefully " }
-{ "l_orderkey": 1671, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50470.74, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". slyly bold instructions boost. furiousl" }
-{ "l_orderkey": 2789, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cording to the careful de" }
-{ "l_orderkey": 2820, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g multipliers. final c" }
-{ "l_orderkey": 3138, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal foxes affix slyly. fluffily regul" }
-{ "l_orderkey": 3616, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly ironic accounts unwind b" }
-{ "l_orderkey": 3748, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans run carefully quic" }
-{ "l_orderkey": 5056, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodolites. ironic a" }
-{ "l_orderkey": 5536, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38401.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "c, final theo" }
+{ "l_orderkey": 548, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-11-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests haggle quickly eve" }
+{ "l_orderkey": 706, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ckey players. requests above the" }
+{ "l_orderkey": 961, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warhorses slee" }
 { "l_orderkey": 1283, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 23040.99, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "fully regular " }
 { "l_orderkey": 1378, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37304.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le furiously slyly final accounts. careful" }
+{ "l_orderkey": 1671, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50470.74, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". slyly bold instructions boost. furiousl" }
+{ "l_orderkey": 1892, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15360.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously about the furiously" }
 { "l_orderkey": 2080, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic deposits haggle slyly carefully eve" }
-{ "l_orderkey": 2882, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31818.51, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. furiously ironic" }
-{ "l_orderkey": 5313, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17555.04, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ckages wake carefully aga" }
-{ "l_orderkey": 5509, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3291.57, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " quickly fin" }
-{ "l_orderkey": 70, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14263.47, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly special packag" }
-{ "l_orderkey": 258, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "silent frets nod daringly busy, bold" }
-{ "l_orderkey": 706, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ckey players. requests above the" }
 { "l_orderkey": 2180, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uriously f" }
+{ "l_orderkey": 2789, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cording to the careful de" }
 { "l_orderkey": 2790, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-12-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ments. slyly f" }
+{ "l_orderkey": 2820, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g multipliers. final c" }
+{ "l_orderkey": 2882, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31818.51, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. furiously ironic" }
+{ "l_orderkey": 3138, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal foxes affix slyly. fluffily regul" }
+{ "l_orderkey": 3296, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic ideas across the" }
 { "l_orderkey": 3552, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19749.42, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s deposits against the blithely unusual pin" }
+{ "l_orderkey": 3587, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5485.95, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely regular decoys above the " }
+{ "l_orderkey": 3616, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly ironic accounts unwind b" }
+{ "l_orderkey": 3748, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans run carefully quic" }
 { "l_orderkey": 3969, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28526.94, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily; braids detect." }
+{ "l_orderkey": 4064, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es boost. careful" }
 { "l_orderkey": 4194, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 47179.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-06", "l_commitdate": "1994-12-09", "l_receiptdate": "1994-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "olites are after the exp" }
 { "l_orderkey": 4773, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39498.84, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-03-03", "l_receiptdate": "1996-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " dependencies. quickly" }
-{ "l_orderkey": 548, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-11-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests haggle quickly eve" }
-{ "l_orderkey": 961, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warhorses slee" }
-{ "l_orderkey": 1892, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15360.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously about the furiously" }
-{ "l_orderkey": 3296, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic ideas across the" }
-{ "l_orderkey": 3587, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5485.95, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely regular decoys above the " }
-{ "l_orderkey": 4064, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es boost. careful" }
+{ "l_orderkey": 5056, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodolites. ironic a" }
 { "l_orderkey": 5185, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40596.03, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gainst the courts dazzle care" }
-{ "l_orderkey": 1731, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25212.37, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rays? bold, express pac" }
-{ "l_orderkey": 4000, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-03-14", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ve the even, fi" }
-{ "l_orderkey": 4038, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43847.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t. slyly silent pinto beans amo" }
-{ "l_orderkey": 4289, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20827.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e carefully regular ideas. sl" }
-{ "l_orderkey": 4583, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to beans haggle sly" }
-{ "l_orderkey": 4803, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 46039.98, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " accounts affix quickly ar" }
+{ "l_orderkey": 5313, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17555.04, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ckages wake carefully aga" }
+{ "l_orderkey": 5509, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3291.57, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " quickly fin" }
+{ "l_orderkey": 5536, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38401.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "c, final theo" }
 { "l_orderkey": 71, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 37270.46, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s cajole. " }
-{ "l_orderkey": 1060, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously. furiously regular in" }
-{ "l_orderkey": 1539, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 23019.99, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts haggle. busy" }
-{ "l_orderkey": 2214, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "t the blithely" }
-{ "l_orderkey": 2306, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-27", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly " }
-{ "l_orderkey": 3106, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6577.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "symptotes. slyly bold platelets cajol" }
-{ "l_orderkey": 4613, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 51520.93, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously special requests wak" }
-{ "l_orderkey": 5922, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9865.71, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "haggle slyly even packages. packages" }
-{ "l_orderkey": 5959, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14250.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar forges. deposits det" }
-{ "l_orderkey": 5986, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 27404.75, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions. slyly regular de" }
 { "l_orderkey": 259, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3288.57, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng slyly at the accounts." }
 { "l_orderkey": 482, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tructions near the final, regular ideas de" }
 { "l_orderkey": 549, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19731.42, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "furiously according to the ironic, regular " }
 { "l_orderkey": 614, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32885.7, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-02-08", "l_receiptdate": "1993-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "tructions are f" }
-{ "l_orderkey": 1828, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12058.09, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " wake blithely " }
-{ "l_orderkey": 2115, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-07-29", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully bold accounts " }
-{ "l_orderkey": 3717, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49328.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s the blithely unu" }
-{ "l_orderkey": 3845, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "kages. care" }
-{ "l_orderkey": 4034, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7673.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y even theodolites. slyly regular instru" }
-{ "l_orderkey": 4128, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5480.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ake permanently " }
-{ "l_orderkey": 4230, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-11", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ar packages are " }
-{ "l_orderkey": 4390, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-06-22", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld braids haggle atop the for" }
-{ "l_orderkey": 4736, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28500.94, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1996-01-18", "l_receiptdate": "1996-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "efully speci" }
-{ "l_orderkey": 4902, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r the furiously final fox" }
 { "l_orderkey": 768, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "out the ironic" }
+{ "l_orderkey": 1060, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously. furiously regular in" }
 { "l_orderkey": 1154, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " even, special " }
 { "l_orderkey": 1155, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly final pinto beans was." }
+{ "l_orderkey": 1539, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 23019.99, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts haggle. busy" }
+{ "l_orderkey": 1731, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25212.37, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rays? bold, express pac" }
+{ "l_orderkey": 1828, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12058.09, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " wake blithely " }
+{ "l_orderkey": 2115, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-07-29", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully bold accounts " }
+{ "l_orderkey": 2214, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "t the blithely" }
+{ "l_orderkey": 2306, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-27", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly " }
+{ "l_orderkey": 3106, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6577.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "symptotes. slyly bold platelets cajol" }
+{ "l_orderkey": 3717, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49328.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s the blithely unu" }
+{ "l_orderkey": 3845, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "kages. care" }
+{ "l_orderkey": 4000, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-03-14", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ve the even, fi" }
+{ "l_orderkey": 4034, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7673.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y even theodolites. slyly regular instru" }
+{ "l_orderkey": 4038, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43847.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t. slyly silent pinto beans amo" }
+{ "l_orderkey": 4128, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5480.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ake permanently " }
+{ "l_orderkey": 4230, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-11", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ar packages are " }
 { "l_orderkey": 4263, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ideas for the carefully re" }
+{ "l_orderkey": 4289, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20827.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e carefully regular ideas. sl" }
+{ "l_orderkey": 4390, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-06-22", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld braids haggle atop the for" }
+{ "l_orderkey": 4583, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to beans haggle sly" }
+{ "l_orderkey": 4613, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 51520.93, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously special requests wak" }
+{ "l_orderkey": 4736, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28500.94, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1996-01-18", "l_receiptdate": "1996-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "efully speci" }
+{ "l_orderkey": 4803, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 46039.98, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " accounts affix quickly ar" }
+{ "l_orderkey": 4902, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r the furiously final fox" }
 { "l_orderkey": 5185, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly blithe deposits. furi" }
 { "l_orderkey": 5637, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-11", "l_receiptdate": "1996-09-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ickly ironic gifts. blithely even cour" }
-{ "l_orderkey": 1156, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 45997.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1997-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "even requests boost ironic deposits. pe" }
-{ "l_orderkey": 1667, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2190.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "pecial requests hag" }
-{ "l_orderkey": 1702, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50378.74, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-07-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y even foxes. carefully final dependencies " }
-{ "l_orderkey": 2439, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36141.27, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "asymptotes wake packages-- furiously" }
-{ "l_orderkey": 2848, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19713.42, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "osits haggle. stealthily ironic packa" }
-{ "l_orderkey": 3744, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32855.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nts among " }
+{ "l_orderkey": 5922, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9865.71, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "haggle slyly even packages. packages" }
+{ "l_orderkey": 5959, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14250.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar forges. deposits det" }
+{ "l_orderkey": 5986, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 27404.75, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions. slyly regular de" }
+{ "l_orderkey": 103, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6571.14, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-11", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-10-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "cajole. carefully ex" }
 { "l_orderkey": 230, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6571.14, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-25", "l_receiptdate": "1994-02-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sleep furiously about the p" }
 { "l_orderkey": 612, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30665.32, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1992-12-11", "l_receiptdate": "1993-01-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular instructions affix bl" }
 { "l_orderkey": 614, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22998.99, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-01-06", "l_receiptdate": "1993-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "arefully. slyly express packag" }
-{ "l_orderkey": 3012, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-07", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly furious packages. silently unusua" }
-{ "l_orderkey": 3173, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38331.65, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " across the slyly even requests." }
-{ "l_orderkey": 5731, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly unusual ideas above the " }
-{ "l_orderkey": 103, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6571.14, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-11", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-10-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "cajole. carefully ex" }
+{ "l_orderkey": 1156, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 45997.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1997-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "even requests boost ironic deposits. pe" }
 { "l_orderkey": 1506, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 16427.85, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-10-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully fluffy packages-- caref" }
+{ "l_orderkey": 1667, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2190.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "pecial requests hag" }
+{ "l_orderkey": 1702, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50378.74, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-07-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y even foxes. carefully final dependencies " }
 { "l_orderkey": 2018, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2190.38, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-07-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly ironic accounts against the slyly sly" }
 { "l_orderkey": 2241, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41617.22, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-31", "l_receiptdate": "1993-08-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " silent, unusual d" }
-{ "l_orderkey": 2690, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 13142.28, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nal, regular atta" }
-{ "l_orderkey": 2883, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39426.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-02", "l_commitdate": "1995-03-14", "l_receiptdate": "1995-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests detect slyly special packages" }
-{ "l_orderkey": 3205, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly pending packages snooz" }
-{ "l_orderkey": 3397, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8761.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y final foxes" }
-{ "l_orderkey": 3648, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "egular instructions. slyly regular pinto" }
-{ "l_orderkey": 5252, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9856.71, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "x. slyly special depos" }
-{ "l_orderkey": 5857, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54759.5, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-12-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y regular d" }
 { "l_orderkey": 2309, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22998.99, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-05", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-11-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "unts around the dolphins ar" }
 { "l_orderkey": 2341, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8761.52, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns affix above the iron" }
+{ "l_orderkey": 2439, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36141.27, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "asymptotes wake packages-- furiously" }
 { "l_orderkey": 2468, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 48188.36, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular, silent sheave" }
+{ "l_orderkey": 2690, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 13142.28, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nal, regular atta" }
+{ "l_orderkey": 2848, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19713.42, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "osits haggle. stealthily ironic packa" }
+{ "l_orderkey": 2883, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39426.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-02", "l_commitdate": "1995-03-14", "l_receiptdate": "1995-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests detect slyly special packages" }
+{ "l_orderkey": 3012, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-07", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly furious packages. silently unusua" }
+{ "l_orderkey": 3173, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38331.65, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " across the slyly even requests." }
+{ "l_orderkey": 3205, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly pending packages snooz" }
 { "l_orderkey": 3239, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28474.94, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-21", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ngly pending platelets are fluff" }
 { "l_orderkey": 3362, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 44902.79, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ake alongside of the " }
+{ "l_orderkey": 3397, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8761.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y final foxes" }
+{ "l_orderkey": 3648, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53664.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "egular instructions. slyly regular pinto" }
+{ "l_orderkey": 3744, "l_partkey": 195, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32855.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nts among " }
 { "l_orderkey": 4135, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14237.47, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-16", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "efully special account" }
 { "l_orderkey": 4355, "l_partkey": 195, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 35046.08, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-29", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y silent deposits. b" }
+{ "l_orderkey": 5252, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9856.71, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "x. slyly special depos" }
+{ "l_orderkey": 5731, "l_partkey": 195, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20808.61, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly unusual ideas above the " }
+{ "l_orderkey": 5857, "l_partkey": 195, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54759.5, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-12-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y regular d" }
+{ "l_orderkey": 354, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26260.56, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent requests. regular, even accounts" }
+{ "l_orderkey": 963, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. slyly regular depe" }
+{ "l_orderkey": 1255, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50332.74, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-08-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons nag qui" }
+{ "l_orderkey": 1378, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31731.51, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ual packages are furiously blith" }
+{ "l_orderkey": 1536, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5470.95, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-08", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "requests sleep pe" }
+{ "l_orderkey": 1573, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 12036.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-23", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "nently pending" }
+{ "l_orderkey": 1954, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53615.31, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eans. final pinto beans sleep furiousl" }
+{ "l_orderkey": 2084, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 37202.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully ironic requests. fluffil" }
+{ "l_orderkey": 2214, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54709.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-06", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts. blith" }
+{ "l_orderkey": 2657, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 33919.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-27", "l_receiptdate": "1995-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re blithely " }
+{ "l_orderkey": 2917, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-03-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly about the regular accounts. carefully pe" }
+{ "l_orderkey": 3073, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17507.04, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "n requests. ironi" }
+{ "l_orderkey": 3174, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4376.76, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas sleep thi" }
+{ "l_orderkey": 3586, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2188.38, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he even, unusual decoy" }
 { "l_orderkey": 3590, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 48144.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-15", "l_receiptdate": "1995-06-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s sleep after the regular platelets. blit" }
 { "l_orderkey": 3716, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42673.41, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "even deposits." }
-{ "l_orderkey": 5062, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27354.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-15", "l_commitdate": "1992-11-17", "l_receiptdate": "1993-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uthless excuses ag" }
-{ "l_orderkey": 1536, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5470.95, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-08", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "requests sleep pe" }
-{ "l_orderkey": 1954, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 53615.31, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eans. final pinto beans sleep furiousl" }
-{ "l_orderkey": 3174, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4376.76, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas sleep thi" }
+{ "l_orderkey": 3842, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30637.32, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lly alongside of the" }
 { "l_orderkey": 3942, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5470.95, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-27", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". fluffily pending deposits above the flu" }
+{ "l_orderkey": 4195, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20789.61, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-08-13", "l_receiptdate": "1993-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "telets sleep even requests. final, even i" }
+{ "l_orderkey": 4355, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15318.66, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-01-22", "l_receiptdate": "1997-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he furiously ironic accounts. quickly iro" }
 { "l_orderkey": 4356, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38296.65, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-30", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "arefully ironic " }
 { "l_orderkey": 4642, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 12036.09, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lithely express asympt" }
 { "l_orderkey": 4803, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20789.61, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-03-25", "l_receiptdate": "1996-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sts. enticing, even" }
 { "l_orderkey": 4931, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1094.19, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1994-12-19", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " furiously " }
 { "l_orderkey": 4966, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6565.14, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1996-11-29", "l_receiptdate": "1996-12-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "d deposits are sly excuses. slyly iro" }
-{ "l_orderkey": 5409, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29543.13, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites " }
-{ "l_orderkey": 1255, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50332.74, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-08-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons nag qui" }
-{ "l_orderkey": 1573, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 12036.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-23", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "nently pending" }
-{ "l_orderkey": 2657, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 33919.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-27", "l_receiptdate": "1995-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re blithely " }
-{ "l_orderkey": 354, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26260.56, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent requests. regular, even accounts" }
-{ "l_orderkey": 963, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. slyly regular depe" }
-{ "l_orderkey": 1378, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31731.51, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ual packages are furiously blith" }
-{ "l_orderkey": 2084, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 37202.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully ironic requests. fluffil" }
-{ "l_orderkey": 2214, "l_partkey": 194, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54709.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-06", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts. blith" }
-{ "l_orderkey": 2917, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7659.33, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-03-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly about the regular accounts. carefully pe" }
-{ "l_orderkey": 3073, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17507.04, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "n requests. ironi" }
-{ "l_orderkey": 3586, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2188.38, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he even, unusual decoy" }
-{ "l_orderkey": 3842, "l_partkey": 194, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30637.32, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lly alongside of the" }
-{ "l_orderkey": 4195, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20789.61, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-08-13", "l_receiptdate": "1993-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "telets sleep even requests. final, even i" }
-{ "l_orderkey": 4355, "l_partkey": 194, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15318.66, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-01-22", "l_receiptdate": "1997-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he furiously ironic accounts. quickly iro" }
+{ "l_orderkey": 5062, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27354.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-15", "l_commitdate": "1992-11-17", "l_receiptdate": "1993-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uthless excuses ag" }
 { "l_orderkey": 5188, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39390.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-03-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "packages? blithely s" }
+{ "l_orderkey": 5409, "l_partkey": 194, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29543.13, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites " }
 { "l_orderkey": 163, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5465.95, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " must belie" }
 { "l_orderkey": 259, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1993-12-22", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " requests sleep" }
+{ "l_orderkey": 320, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14211.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-12-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he furiously regular pinto beans. car" }
+{ "l_orderkey": 710, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41541.22, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sts boost fluffily aft" }
 { "l_orderkey": 898, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39354.84, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " after the carefully " }
-{ "l_orderkey": 1440, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-11-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions boost. fluffily regul" }
-{ "l_orderkey": 1923, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 53566.31, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-21", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully expre" }
-{ "l_orderkey": 4359, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34982.08, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-18", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites nag quietly caref" }
-{ "l_orderkey": 4837, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-06-18", "l_receiptdate": "1998-08-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "counts cajole slyly furiou" }
+{ "l_orderkey": 1057, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31702.51, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-05", "l_commitdate": "1992-05-05", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es wake according to the q" }
 { "l_orderkey": 1254, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-02-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely even deposits eat!" }
 { "l_orderkey": 1383, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15304.66, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ole carefully silent requests. car" }
+{ "l_orderkey": 1440, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-11-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions boost. fluffily regul" }
+{ "l_orderkey": 1696, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22956.99, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-02-18", "l_receiptdate": "1998-05-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y players sleep along the final, pending " }
+{ "l_orderkey": 1923, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 53566.31, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-21", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully expre" }
+{ "l_orderkey": 2180, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-29", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ep furiously furiously final request" }
+{ "l_orderkey": 2244, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-12", "l_commitdate": "1993-03-09", "l_receiptdate": "1993-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "rate around the reques" }
+{ "l_orderkey": 2336, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21863.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-03-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the fi" }
 { "l_orderkey": 2340, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22956.99, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-03-04", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " asymptotes. unusual theo" }
 { "l_orderkey": 2403, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 29516.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "deposits sleep slyly special theodolit" }
+{ "l_orderkey": 3937, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "into beans. slyly silent orbits alongside o" }
+{ "l_orderkey": 4295, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "telets cajole bravely" }
+{ "l_orderkey": 4359, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34982.08, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-18", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites nag quietly caref" }
+{ "l_orderkey": 4837, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-06-18", "l_receiptdate": "1998-08-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "counts cajole slyly furiou" }
+{ "l_orderkey": 4964, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 24050.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-09-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "equests doubt quickly. caref" }
 { "l_orderkey": 5058, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-09", "l_receiptdate": "1998-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " the special foxes " }
 { "l_orderkey": 5285, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33888.89, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-04-05", "l_receiptdate": "1994-05-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ubt. quickly blithe " }
-{ "l_orderkey": 320, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14211.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-12-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he furiously regular pinto beans. car" }
-{ "l_orderkey": 1057, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31702.51, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-05", "l_commitdate": "1992-05-05", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es wake according to the q" }
-{ "l_orderkey": 2244, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-12", "l_commitdate": "1993-03-09", "l_receiptdate": "1993-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "rate around the reques" }
-{ "l_orderkey": 4295, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3279.57, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "telets cajole bravely" }
-{ "l_orderkey": 5666, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-13", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the even, final foxes. quickly iron" }
-{ "l_orderkey": 710, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41541.22, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sts boost fluffily aft" }
-{ "l_orderkey": 1696, "l_partkey": 193, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22956.99, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-02-18", "l_receiptdate": "1998-05-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y players sleep along the final, pending " }
-{ "l_orderkey": 2180, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-29", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ep furiously furiously final request" }
-{ "l_orderkey": 2336, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21863.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-03-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the fi" }
-{ "l_orderkey": 3937, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6559.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "into beans. slyly silent orbits alongside o" }
-{ "l_orderkey": 4964, "l_partkey": 193, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 24050.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-09-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "equests doubt quickly. caref" }
 { "l_orderkey": 5477, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17491.04, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-07", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "regular, s" }
+{ "l_orderkey": 5666, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-13", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the even, final foxes. quickly iron" }
 { "l_orderkey": 5795, "l_partkey": 193, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37168.46, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-07-30", "l_receiptdate": "1992-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al instructions must affix along the ironic" }
 { "l_orderkey": 5954, "l_partkey": 193, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 42634.41, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " always regular dolphins. furiously p" }
-{ "l_orderkey": 742, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17475.04, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites haggle carefully regul" }
-{ "l_orderkey": 3169, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 13106.28, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-05", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular d" }
-{ "l_orderkey": 3523, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39318.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. fluffily regu" }
-{ "l_orderkey": 4291, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3276.57, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-21", "l_receiptdate": "1994-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tes sleep slyly above the quickly sl" }
-{ "l_orderkey": 4582, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18567.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-17", "l_commitdate": "1996-08-26", "l_receiptdate": "1996-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng packages. depo" }
-{ "l_orderkey": 5857, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15290.66, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily pendin" }
 { "l_orderkey": 262, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42595.41, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-02-18", "l_receiptdate": "1996-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual, regular requests" }
 { "l_orderkey": 679, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9829.71, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1996-01-27", "l_receiptdate": "1996-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep slyly. entici" }
+{ "l_orderkey": 742, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17475.04, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites haggle carefully regul" }
 { "l_orderkey": 742, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 53517.31, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " carefully bold foxes sle" }
-{ "l_orderkey": 1445, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7645.33, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-25", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-05-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "structions: slyly regular re" }
-{ "l_orderkey": 2339, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24028.18, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-01-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " furiously above " }
-{ "l_orderkey": 3685, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42595.41, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-19", "l_commitdate": "1992-04-06", "l_receiptdate": "1992-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic courts nag carefully after the " }
-{ "l_orderkey": 5254, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34950.08, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts impress closely furi" }
-{ "l_orderkey": 5345, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50240.74, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-27", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "slyly special deposits. fin" }
-{ "l_orderkey": 5346, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y. fluffily bold accounts grow. furio" }
-{ "l_orderkey": 1283, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43687.6, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "riously. even, ironic instructions after" }
-{ "l_orderkey": 3174, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-26", "l_receiptdate": "1996-02-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "leep quickly? slyly special platelets" }
-{ "l_orderkey": 4454, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49148.55, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests promise. packages print fur" }
-{ "l_orderkey": 4513, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, final excuses detect furi" }
-{ "l_orderkey": 5250, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29489.13, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l forges are. furiously unusual pin" }
-{ "l_orderkey": 5381, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s after the f" }
-{ "l_orderkey": 5731, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-23", "l_receiptdate": "1997-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ngside of the quickly regular depos" }
 { "l_orderkey": 743, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22935.99, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "d requests. packages afte" }
 { "l_orderkey": 1153, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 5460.95, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "special excuses promi" }
 { "l_orderkey": 1255, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 13106.28, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular, express accounts are " }
+{ "l_orderkey": 1283, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43687.6, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "riously. even, ironic instructions after" }
+{ "l_orderkey": 1445, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7645.33, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-25", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-05-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "structions: slyly regular re" }
 { "l_orderkey": 1538, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29489.13, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-09-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ngly even packag" }
+{ "l_orderkey": 2339, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 24028.18, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-01-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " furiously above " }
 { "l_orderkey": 2500, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43687.6, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully unusual dolphins s" }
 { "l_orderkey": 2593, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 12014.09, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-01", "l_commitdate": "1993-11-19", "l_receiptdate": "1993-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "express packages sleep bold re" }
+{ "l_orderkey": 3169, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 13106.28, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-05", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular d" }
+{ "l_orderkey": 3174, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-26", "l_receiptdate": "1996-02-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "leep quickly? slyly special platelets" }
+{ "l_orderkey": 3523, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39318.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. fluffily regu" }
 { "l_orderkey": 3554, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44779.79, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-08-28", "l_receiptdate": "1995-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ent dependencies. sly" }
-{ "l_orderkey": 1668, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9820.71, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-17", "l_commitdate": "1997-09-05", "l_receiptdate": "1997-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "wake furiously even instructions. sil" }
-{ "l_orderkey": 2373, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-05-19", "l_receiptdate": "1994-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "auternes. blithely even pinto bea" }
-{ "l_orderkey": 2531, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39282.84, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-06-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic, bold packages. blithely e" }
-{ "l_orderkey": 3809, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es detect furiously sil" }
-{ "l_orderkey": 3909, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50194.74, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-08", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-10-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "the blithely unusual ideas" }
-{ "l_orderkey": 4485, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-02-07", "l_receiptdate": "1994-12-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "play according to the ironic, ironic" }
-{ "l_orderkey": 4737, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s. fluffily regular " }
-{ "l_orderkey": 4868, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-04-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ys engage. th" }
-{ "l_orderkey": 5952, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 12003.09, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-06-04", "l_receiptdate": "1997-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y nag blithely aga" }
+{ "l_orderkey": 3685, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42595.41, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-19", "l_commitdate": "1992-04-06", "l_receiptdate": "1992-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic courts nag carefully after the " }
+{ "l_orderkey": 4291, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3276.57, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-21", "l_receiptdate": "1994-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tes sleep slyly above the quickly sl" }
+{ "l_orderkey": 4454, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49148.55, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests promise. packages print fur" }
+{ "l_orderkey": 4513, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, final excuses detect furi" }
+{ "l_orderkey": 4582, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18567.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-17", "l_commitdate": "1996-08-26", "l_receiptdate": "1996-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng packages. depo" }
+{ "l_orderkey": 5250, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29489.13, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l forges are. furiously unusual pin" }
+{ "l_orderkey": 5254, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34950.08, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts impress closely furi" }
+{ "l_orderkey": 5345, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50240.74, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-27", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "slyly special deposits. fin" }
+{ "l_orderkey": 5346, "l_partkey": 192, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y. fluffily bold accounts grow. furio" }
+{ "l_orderkey": 5381, "l_partkey": 192, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s after the f" }
+{ "l_orderkey": 5731, "l_partkey": 192, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 14198.47, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-23", "l_receiptdate": "1997-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ngside of the quickly regular depos" }
+{ "l_orderkey": 5857, "l_partkey": 192, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15290.66, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily pendin" }
 { "l_orderkey": 163, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21823.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "tructions integrate b" }
 { "l_orderkey": 358, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44738.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ely frets. furious deposits sleep " }
+{ "l_orderkey": 550, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33826.89, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "thely silent packages. unusual" }
 { "l_orderkey": 993, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43647.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gle above the furiously " }
+{ "l_orderkey": 1031, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 48012.36, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "re slyly above the furio" }
 { "l_orderkey": 1506, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30553.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " unwind carefully: theodolit" }
+{ "l_orderkey": 1574, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54559.5, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1997-02-14", "l_receiptdate": "1996-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "le regular, regular foxes. blithely e" }
 { "l_orderkey": 1574, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6547.14, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-03-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e silent, final packages. speci" }
+{ "l_orderkey": 1632, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 51285.93, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g to the closely special no" }
+{ "l_orderkey": 1668, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9820.71, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-17", "l_commitdate": "1997-09-05", "l_receiptdate": "1997-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "wake furiously even instructions. sil" }
+{ "l_orderkey": 1792, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 49103.55, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1993-12-24", "l_receiptdate": "1994-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests are. ironic, regular asy" }
+{ "l_orderkey": 1859, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lar packages wake quickly exp" }
+{ "l_orderkey": 2176, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41465.22, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1993-01-14", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lithely ironic pinto beans. furious" }
+{ "l_orderkey": 2373, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-05-19", "l_receiptdate": "1994-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "auternes. blithely even pinto bea" }
+{ "l_orderkey": 2531, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 39282.84, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-06-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic, bold packages. blithely e" }
+{ "l_orderkey": 3265, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30553.32, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n requests. quickly final dinos" }
+{ "l_orderkey": 3298, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully regular requ" }
 { "l_orderkey": 3361, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 33826.89, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-09-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts. pending, regular accounts sleep fur" }
+{ "l_orderkey": 3363, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-28", "l_receiptdate": "1995-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he regular, brave deposits. f" }
+{ "l_orderkey": 3590, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts above the silent waters thrash f" }
+{ "l_orderkey": 3809, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18550.23, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es detect furiously sil" }
+{ "l_orderkey": 3909, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 50194.74, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-08", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-10-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "the blithely unusual ideas" }
+{ "l_orderkey": 3971, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2182.38, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-15", "l_commitdate": "1996-08-12", "l_receiptdate": "1996-07-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "haggle abou" }
 { "l_orderkey": 4421, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34918.08, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-09", "l_commitdate": "1997-06-03", "l_receiptdate": "1997-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar ideas eat among the furiousl" }
+{ "l_orderkey": 4485, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-02-07", "l_receiptdate": "1994-12-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "play according to the ironic, ironic" }
+{ "l_orderkey": 4519, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly slyly furious depth" }
+{ "l_orderkey": 4646, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26188.56, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic platelets lose carefully. blithely unu" }
 { "l_orderkey": 4708, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19641.42, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-11", "l_commitdate": "1994-11-15", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "special, eve" }
+{ "l_orderkey": 4737, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s. fluffily regular " }
+{ "l_orderkey": 4868, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-04-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ys engage. th" }
 { "l_orderkey": 5350, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 48012.36, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p above the ironic, pending dep" }
+{ "l_orderkey": 5443, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6547.14, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-17", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p fluffily foxe" }
 { "l_orderkey": 5669, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7638.33, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-07-07", "l_receiptdate": "1996-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "yly regular requests lose blithely. careful" }
 { "l_orderkey": 5699, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32735.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-13", "l_commitdate": "1992-10-01", "l_receiptdate": "1992-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the carefully final " }
 { "l_orderkey": 5767, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45829.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithe deposi" }
+{ "l_orderkey": 5829, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " ironic excuses use fluf" }
 { "l_orderkey": 5831, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2182.38, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-20", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "quickly silent req" }
 { "l_orderkey": 5859, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29462.13, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across th" }
 { "l_orderkey": 5861, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34918.08, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nt asymptotes. carefully express request" }
-{ "l_orderkey": 550, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33826.89, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "thely silent packages. unusual" }
-{ "l_orderkey": 1574, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 54559.5, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1997-02-14", "l_receiptdate": "1996-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "le regular, regular foxes. blithely e" }
-{ "l_orderkey": 1632, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 51285.93, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g to the closely special no" }
-{ "l_orderkey": 1792, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 49103.55, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1993-12-24", "l_receiptdate": "1994-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests are. ironic, regular asy" }
-{ "l_orderkey": 3265, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30553.32, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n requests. quickly final dinos" }
-{ "l_orderkey": 3298, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1091.19, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully regular requ" }
-{ "l_orderkey": 3971, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2182.38, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-15", "l_commitdate": "1996-08-12", "l_receiptdate": "1996-07-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "haggle abou" }
-{ "l_orderkey": 1031, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 48012.36, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "re slyly above the furio" }
-{ "l_orderkey": 1859, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lar packages wake quickly exp" }
-{ "l_orderkey": 2176, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41465.22, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1993-01-14", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lithely ironic pinto beans. furious" }
-{ "l_orderkey": 3363, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22914.99, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-28", "l_receiptdate": "1995-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he regular, brave deposits. f" }
-{ "l_orderkey": 3590, "l_partkey": 191, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts above the silent waters thrash f" }
-{ "l_orderkey": 4519, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 40374.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly slyly furious depth" }
-{ "l_orderkey": 4646, "l_partkey": 191, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26188.56, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic platelets lose carefully. blithely unu" }
-{ "l_orderkey": 5443, "l_partkey": 191, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6547.14, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-17", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p fluffily foxe" }
-{ "l_orderkey": 5829, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 53468.31, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " ironic excuses use fluf" }
+{ "l_orderkey": 5952, "l_partkey": 191, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 12003.09, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-06-04", "l_receiptdate": "1997-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y nag blithely aga" }
+{ "l_orderkey": 131, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4360.76, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " are carefully slyly i" }
 { "l_orderkey": 162, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es! final somas integrate" }
+{ "l_orderkey": 224, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44697.79, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "after the furiou" }
+{ "l_orderkey": 358, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-18", "l_commitdate": "1993-12-12", "l_receiptdate": "1993-10-31", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y final foxes sleep blithely sl" }
+{ "l_orderkey": 389, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "fts. courts eat blithely even dependenc" }
+{ "l_orderkey": 610, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-19", "l_receiptdate": "1995-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " ironic pinto beans haggle. blithe" }
 { "l_orderkey": 643, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 51238.93, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-05", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y against " }
 { "l_orderkey": 672, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9811.71, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-25", "l_commitdate": "1994-06-06", "l_receiptdate": "1994-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "haggle carefully carefully reg" }
+{ "l_orderkey": 704, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43607.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ggle quickly. r" }
 { "l_orderkey": 871, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31615.51, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1996-01-27", "l_receiptdate": "1995-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ests are carefu" }
+{ "l_orderkey": 1059, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54509.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-15", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s impress furiously about" }
+{ "l_orderkey": 1185, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 13082.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-26", "l_receiptdate": "1992-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "instructions. daringly pend" }
+{ "l_orderkey": 1344, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31615.51, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ffily quiet foxes wake blithely. slyly " }
+{ "l_orderkey": 1510, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39246.84, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old deposits along the carefully" }
+{ "l_orderkey": 1607, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages haggle. regular requests boost s" }
 { "l_orderkey": 1856, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15262.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-14", "l_commitdate": "1992-05-02", "l_receiptdate": "1992-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ans are even requests. deposits caj" }
+{ "l_orderkey": 2182, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3270.57, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y bold theodolites wi" }
+{ "l_orderkey": 2437, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28344.94, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-07-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly regular accounts." }
+{ "l_orderkey": 2885, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5450.95, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-13", "l_receiptdate": "1993-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s. slyly express th" }
+{ "l_orderkey": 3334, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nts sublate slyly express pack" }
+{ "l_orderkey": 3622, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50148.74, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits wake. blithe" }
 { "l_orderkey": 3780, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43607.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-06", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gular deposits-- furiously regular " }
 { "l_orderkey": 4034, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 52329.12, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " blithely regular requests play carefull" }
 { "l_orderkey": 4327, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 42517.41, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "kages against the blit" }
 { "l_orderkey": 4421, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-08-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly final pinto beans impress. bold " }
+{ "l_orderkey": 4481, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ackages haggle even, " }
+{ "l_orderkey": 4484, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 41427.22, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". even requests un" }
 { "l_orderkey": 4487, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1090.19, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ithely final asym" }
 { "l_orderkey": 4641, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49058.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " about the close " }
-{ "l_orderkey": 358, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-18", "l_commitdate": "1993-12-12", "l_receiptdate": "1993-10-31", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y final foxes sleep blithely sl" }
-{ "l_orderkey": 389, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "fts. courts eat blithely even dependenc" }
-{ "l_orderkey": 610, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-19", "l_receiptdate": "1995-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " ironic pinto beans haggle. blithe" }
-{ "l_orderkey": 704, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43607.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ggle quickly. r" }
-{ "l_orderkey": 1344, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31615.51, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ffily quiet foxes wake blithely. slyly " }
-{ "l_orderkey": 1510, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39246.84, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old deposits along the carefully" }
-{ "l_orderkey": 4481, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ackages haggle even, " }
+{ "l_orderkey": 4679, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. bold, regular packa" }
 { "l_orderkey": 4773, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11992.09, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-02", "l_commitdate": "1996-01-29", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "en accounts. slyly b" }
 { "l_orderkey": 4807, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-04", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "efully even dolphins slee" }
-{ "l_orderkey": 5926, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25074.37, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-23", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ickly special packages among " }
-{ "l_orderkey": 5984, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 38156.65, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "le fluffily regula" }
-{ "l_orderkey": 1059, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54509.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-15", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s impress furiously about" }
-{ "l_orderkey": 1185, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 13082.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-26", "l_receiptdate": "1992-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "instructions. daringly pend" }
-{ "l_orderkey": 2885, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5450.95, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-13", "l_receiptdate": "1993-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s. slyly express th" }
-{ "l_orderkey": 3622, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 50148.74, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits wake. blithe" }
-{ "l_orderkey": 5154, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11992.09, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "luffily bold foxes. final" }
-{ "l_orderkey": 131, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4360.76, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " are carefully slyly i" }
-{ "l_orderkey": 224, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44697.79, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "after the furiou" }
-{ "l_orderkey": 1607, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2180.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages haggle. regular requests boost s" }
-{ "l_orderkey": 2182, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3270.57, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y bold theodolites wi" }
-{ "l_orderkey": 2437, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28344.94, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-07-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly regular accounts." }
-{ "l_orderkey": 3334, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nts sublate slyly express pack" }
-{ "l_orderkey": 4484, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 41427.22, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". even requests un" }
-{ "l_orderkey": 4679, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7631.33, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. bold, regular packa" }
 { "l_orderkey": 4930, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 41427.22, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "bold requests sleep never" }
+{ "l_orderkey": 5154, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11992.09, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "luffily bold foxes. final" }
 { "l_orderkey": 5413, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 34886.08, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1998-01-03", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully special package" }
 { "l_orderkey": 5446, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29435.13, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ously across the quic" }
 { "l_orderkey": 5763, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 9811.71, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-09-01", "l_receiptdate": "1998-10-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " deposits. instru" }
-{ "l_orderkey": 512, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20694.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " sleep. requests alongside of the fluff" }
-{ "l_orderkey": 839, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51191.46, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-06", "l_receiptdate": "1995-11-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully final excuses about " }
-{ "l_orderkey": 2725, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16337.7, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "? furiously regular a" }
-{ "l_orderkey": 3847, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7624.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-06", "l_receiptdate": "1993-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " about the blithely daring Tiresias. fl" }
-{ "l_orderkey": 4448, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14159.34, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-26", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "fluffily express accounts integrate furiou" }
-{ "l_orderkey": 4674, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 38121.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le quickly after the express sent" }
-{ "l_orderkey": 4803, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22872.78, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-15", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " silent packages use. b" }
+{ "l_orderkey": 5926, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25074.37, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-23", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ickly special packages among " }
+{ "l_orderkey": 5984, "l_partkey": 190, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 38156.65, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "le fluffily regula" }
 { "l_orderkey": 134, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 28318.68, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " among the pending depos" }
+{ "l_orderkey": 512, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20694.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " sleep. requests alongside of the fluff" }
+{ "l_orderkey": 549, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the regular, furious excuses. carefu" }
 { "l_orderkey": 583, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14159.34, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-23", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y sly theodolites. ironi" }
+{ "l_orderkey": 612, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 35942.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-01", "l_receiptdate": "1992-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "bove the blithely even ideas. careful" }
+{ "l_orderkey": 705, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50102.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss deposits. ironic packa" }
+{ "l_orderkey": 839, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51191.46, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-06", "l_receiptdate": "1995-11-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully final excuses about " }
 { "l_orderkey": 1280, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6535.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits " }
 { "l_orderkey": 1285, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4356.72, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-08-17", "l_receiptdate": "1992-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l packages sleep slyly quiet i" }
 { "l_orderkey": 1286, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11980.98, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-07-30", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " slyly even packages. requ" }
 { "l_orderkey": 1543, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45745.56, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-11", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "xpress instructions. regular acc" }
-{ "l_orderkey": 2372, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11980.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " silent, pending de" }
-{ "l_orderkey": 2624, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 13070.16, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "er the quickly unu" }
-{ "l_orderkey": 3427, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26140.32, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-07-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y bold, sly deposits. pendi" }
-{ "l_orderkey": 4514, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending excuses. sl" }
-{ "l_orderkey": 5542, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6535.08, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-14", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " foxes doubt. theodolites ca" }
-{ "l_orderkey": 612, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 35942.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-01", "l_receiptdate": "1992-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "bove the blithely even ideas. careful" }
-{ "l_orderkey": 705, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50102.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss deposits. ironic packa" }
+{ "l_orderkey": 2049, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27229.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-31", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " excuses above the " }
 { "l_orderkey": 2245, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15248.52, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-05-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nts. always unusual dep" }
+{ "l_orderkey": 2372, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11980.98, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " silent, pending de" }
+{ "l_orderkey": 2496, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39210.48, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ully ironic f" }
 { "l_orderkey": 2565, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28318.68, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-09", "l_receiptdate": "1998-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " pinto beans about the slyly regula" }
+{ "l_orderkey": 2624, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 13070.16, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "er the quickly unu" }
+{ "l_orderkey": 2690, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3267.54, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-04", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". final reques" }
+{ "l_orderkey": 2725, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16337.7, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "? furiously regular a" }
 { "l_orderkey": 2883, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51191.46, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-02-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ep carefully ironic" }
+{ "l_orderkey": 3270, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31586.22, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sly regular asymptotes. slyly dog" }
+{ "l_orderkey": 3427, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26140.32, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-07-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y bold, sly deposits. pendi" }
 { "l_orderkey": 3430, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2178.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-07", "l_commitdate": "1995-01-28", "l_receiptdate": "1995-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sh furiously according to the evenly e" }
 { "l_orderkey": 3459, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10891.8, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-10-17", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ". blithely ironic pinto beans above" }
+{ "l_orderkey": 3847, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7624.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-06", "l_receiptdate": "1993-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " about the blithely daring Tiresias. fl" }
+{ "l_orderkey": 4448, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14159.34, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-26", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "fluffily express accounts integrate furiou" }
+{ "l_orderkey": 4514, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending excuses. sl" }
+{ "l_orderkey": 4580, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 42478.02, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". fluffily final dolphins use furiously al" }
+{ "l_orderkey": 4674, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 38121.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le quickly after the express sent" }
+{ "l_orderkey": 4803, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22872.78, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-15", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " silent packages use. b" }
 { "l_orderkey": 4805, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 49013.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-07-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously sly t" }
 { "l_orderkey": 5413, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 5445.9, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-28", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-12-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tes are al" }
-{ "l_orderkey": 549, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41388.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the regular, furious excuses. carefu" }
-{ "l_orderkey": 2049, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27229.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-31", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " excuses above the " }
-{ "l_orderkey": 2496, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 39210.48, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ully ironic f" }
-{ "l_orderkey": 2690, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3267.54, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-04", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". final reques" }
-{ "l_orderkey": 3270, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31586.22, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sly regular asymptotes. slyly dog" }
-{ "l_orderkey": 4580, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 42478.02, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". fluffily final dolphins use furiously al" }
+{ "l_orderkey": 5542, "l_partkey": 189, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6535.08, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-14", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " foxes doubt. theodolites ca" }
 { "l_orderkey": 578, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 25028.14, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nstructions. ironic deposits" }
+{ "l_orderkey": 738, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4352.72, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ar packages. fluffily bo" }
 { "l_orderkey": 739, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32645.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-07-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "above the even deposits. ironic requests" }
-{ "l_orderkey": 1859, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39174.48, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular requests. carefully unusual theo" }
-{ "l_orderkey": 3203, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-01", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e the blithely regular accounts boost f" }
-{ "l_orderkey": 4320, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35909.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess asymptotes so" }
-{ "l_orderkey": 5381, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40262.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final deposits print carefully. unusua" }
-{ "l_orderkey": 5511, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al theodolites. blithely final de" }
 { "l_orderkey": 836, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6529.08, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1997-01-31", "l_receiptdate": "1996-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fully bold theodolites are daringly across" }
 { "l_orderkey": 896, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 36998.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-05-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular, close requests cajo" }
 { "l_orderkey": 962, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-09-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "efully bold packages run slyly caref" }
+{ "l_orderkey": 1251, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use quickly final packages. iron" }
 { "l_orderkey": 1285, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42439.02, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-15", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uctions. car" }
+{ "l_orderkey": 1410, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular account" }
+{ "l_orderkey": 1859, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39174.48, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular requests. carefully unusual theo" }
+{ "l_orderkey": 2945, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36998.12, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-02-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "at the unusual theodolite" }
 { "l_orderkey": 2979, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38086.3, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-06-11", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "old ideas beneath the blit" }
 { "l_orderkey": 3079, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2176.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-10-25", "l_receiptdate": "1998-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y regular asymptotes doz" }
 { "l_orderkey": 3169, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 13058.16, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-18", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "atelets. pac" }
+{ "l_orderkey": 3203, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-01", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e the blithely regular accounts boost f" }
+{ "l_orderkey": 3362, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50056.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-10-12", "l_receiptdate": "1995-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly bold packages. regular deposits cajol" }
+{ "l_orderkey": 3653, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44615.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic packages affix sly" }
 { "l_orderkey": 3746, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3264.54, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-03", "l_commitdate": "1994-12-10", "l_receiptdate": "1994-11-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the silent ideas cajole carefully " }
+{ "l_orderkey": 3781, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42439.02, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts are carefully. ir" }
+{ "l_orderkey": 3936, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26116.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1997-01-01", "l_receiptdate": "1996-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns. accounts mold fl" }
+{ "l_orderkey": 4226, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29380.86, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-03", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly alongside of the slyly ironic pac" }
+{ "l_orderkey": 4320, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35909.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess asymptotes so" }
 { "l_orderkey": 4547, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16322.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-08", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-12-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ets haggle. regular dinos affix fu" }
 { "l_orderkey": 4742, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 33733.58, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-13", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ke slyly among the furiousl" }
 { "l_orderkey": 4935, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 39174.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "requests across the quick" }
+{ "l_orderkey": 5155, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ole blithely slyly ironic " }
+{ "l_orderkey": 5381, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40262.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final deposits print carefully. unusua" }
+{ "l_orderkey": 5511, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al theodolites. blithely final de" }
+{ "l_orderkey": 5698, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nts. slyly quiet pinto beans nag carefu" }
 { "l_orderkey": 5764, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4352.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-23", "l_receiptdate": "1993-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ily regular courts haggle" }
 { "l_orderkey": 5766, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-16", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular the" }
-{ "l_orderkey": 738, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4352.72, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ar packages. fluffily bo" }
-{ "l_orderkey": 1251, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use quickly final packages. iron" }
-{ "l_orderkey": 3362, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50056.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-10-12", "l_receiptdate": "1995-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly bold packages. regular deposits cajol" }
-{ "l_orderkey": 4226, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29380.86, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-03", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly alongside of the slyly ironic pac" }
-{ "l_orderkey": 1410, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23939.96, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular account" }
-{ "l_orderkey": 2945, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36998.12, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-02-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "at the unusual theodolite" }
-{ "l_orderkey": 3653, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44615.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic packages affix sly" }
-{ "l_orderkey": 3781, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42439.02, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts are carefully. ir" }
-{ "l_orderkey": 3936, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 26116.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1997-01-01", "l_receiptdate": "1996-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns. accounts mold fl" }
-{ "l_orderkey": 5155, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5440.9, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ole blithely slyly ironic " }
-{ "l_orderkey": 5698, "l_partkey": 188, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1088.18, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nts. slyly quiet pinto beans nag carefu" }
 { "l_orderkey": 39, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28266.68, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ckages across the slyly silent" }
+{ "l_orderkey": 293, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11958.98, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " affix carefully quickly special idea" }
 { "l_orderkey": 614, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 52184.64, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-01-19", "l_receiptdate": "1993-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously special excuses haggle along the" }
+{ "l_orderkey": 741, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27179.5, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "accounts. blithely bold pa" }
 { "l_orderkey": 1031, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 29353.86, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gular deposits cajole. blithely unus" }
 { "l_orderkey": 1346, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32615.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-01", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-10-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " nag blithely. unusual, ru" }
+{ "l_orderkey": 1475, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-13", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". slyly bold re" }
 { "l_orderkey": 1509, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 33702.58, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-14", "l_commitdate": "1993-08-21", "l_receiptdate": "1993-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ic deposits cajole carefully. quickly bold " }
+{ "l_orderkey": 1639, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-06", "l_receiptdate": "1995-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the regular packages. courts dou" }
 { "l_orderkey": 2211, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19569.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-31", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-09-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "c grouches. slyly express pinto " }
-{ "l_orderkey": 2980, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-12", "l_commitdate": "1996-10-27", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "elets. fluffily regular in" }
-{ "l_orderkey": 3334, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21743.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses nag furiously. instructions are ca" }
-{ "l_orderkey": 3650, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20656.42, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y even forges. fluffily furious accounts" }
-{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9784.62, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-06-26", "l_receiptdate": "1992-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits serve slyly. unusual pint" }
-{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14133.34, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " wake. unusual platelets for the" }
-{ "l_orderkey": 5600, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36964.12, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly above the stealthy ideas. permane" }
-{ "l_orderkey": 5920, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-01-03", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "across the carefully pending platelets" }
-{ "l_orderkey": 293, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11958.98, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " affix carefully quickly special idea" }
 { "l_orderkey": 2406, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27179.5, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-03", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "al, regular in" }
 { "l_orderkey": 2849, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42400.02, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-07-18", "l_receiptdate": "1996-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s sleep furiously silently regul" }
-{ "l_orderkey": 2951, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43487.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ial deposits wake fluffily about th" }
-{ "l_orderkey": 3558, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3261.54, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-28", "l_receiptdate": "1996-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l, final deposits haggle. fina" }
-{ "l_orderkey": 4583, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46748.74, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-12-17", "l_receiptdate": "1994-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fully after the speci" }
-{ "l_orderkey": 5827, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32615.4, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ounts may c" }
-{ "l_orderkey": 1475, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-13", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". slyly bold re" }
-{ "l_orderkey": 1639, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-06", "l_receiptdate": "1995-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the regular packages. courts dou" }
 { "l_orderkey": 2950, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-09-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ides the b" }
+{ "l_orderkey": 2951, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43487.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ial deposits wake fluffily about th" }
+{ "l_orderkey": 2980, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 26092.32, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-12", "l_commitdate": "1996-10-27", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "elets. fluffily regular in" }
+{ "l_orderkey": 3334, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21743.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses nag furiously. instructions are ca" }
+{ "l_orderkey": 3558, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3261.54, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-28", "l_receiptdate": "1996-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l, final deposits haggle. fina" }
+{ "l_orderkey": 3650, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 20656.42, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y even forges. fluffily furious accounts" }
 { "l_orderkey": 3748, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5435.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-29", "l_commitdate": "1998-05-06", "l_receiptdate": "1998-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " regular accounts sleep quickly-- furious" }
-{ "l_orderkey": 4391, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ep quickly after " }
-{ "l_orderkey": 4930, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38051.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-09", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lose slyly regular dependencies. fur" }
-{ "l_orderkey": 741, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27179.5, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "accounts. blithely bold pa" }
 { "l_orderkey": 3840, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-31", "l_commitdate": "1998-09-19", "l_receiptdate": "1998-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "o beans are. carefully final courts x" }
+{ "l_orderkey": 4391, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 48923.1, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ep quickly after " }
+{ "l_orderkey": 4583, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46748.74, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-12-17", "l_receiptdate": "1994-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fully after the speci" }
 { "l_orderkey": 4647, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2174.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " pinto beans believe furiously slyly silent" }
-{ "l_orderkey": 870, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-09-11", "l_receiptdate": "1993-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly excuses. ironi" }
-{ "l_orderkey": 1670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44533.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al gifts. speci" }
-{ "l_orderkey": 3206, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26068.32, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "encies sleep deposits--" }
-{ "l_orderkey": 3489, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20637.42, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-31", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-08-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "c deposits alongside of the pending, fu" }
-{ "l_orderkey": 5286, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 41274.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-26", "l_receiptdate": "1997-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "fluffily. special, ironic deposit" }
-{ "l_orderkey": 5670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46705.74, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ests in place of the carefully sly depos" }
+{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9784.62, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-06-26", "l_receiptdate": "1992-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits serve slyly. unusual pint" }
+{ "l_orderkey": 4738, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14133.34, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " wake. unusual platelets for the" }
+{ "l_orderkey": 4930, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 38051.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-09", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lose slyly regular dependencies. fur" }
+{ "l_orderkey": 5600, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36964.12, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly above the stealthy ideas. permane" }
+{ "l_orderkey": 5827, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32615.4, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ounts may c" }
+{ "l_orderkey": 5920, "l_partkey": 187, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54359.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-01-03", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "across the carefully pending platelets" }
 { "l_orderkey": 129, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39102.48, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-25", "l_receiptdate": "1992-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "packages are care" }
+{ "l_orderkey": 230, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49964.28, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old packages ha" }
+{ "l_orderkey": 325, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " theodolites. " }
 { "l_orderkey": 357, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39102.48, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-11-13", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d the carefully even requests. " }
 { "l_orderkey": 481, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45619.56, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "mptotes are furiously among the iron" }
 { "l_orderkey": 518, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 52136.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-03-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slyly final platelets; quickly even deposi" }
+{ "l_orderkey": 610, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18465.06, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "p quickly instead of the slyly pending foxe" }
 { "l_orderkey": 613, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3258.54, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ccounts cajole. " }
 { "l_orderkey": 710, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 13034.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-18", "l_commitdate": "1993-02-27", "l_receiptdate": "1993-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ions. slyly express theodolites al" }
+{ "l_orderkey": 870, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-09-11", "l_receiptdate": "1993-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly excuses. ironi" }
+{ "l_orderkey": 1573, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-24", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ymptotes could u" }
+{ "l_orderkey": 1670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44533.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al gifts. speci" }
 { "l_orderkey": 2114, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28240.68, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-30", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-05-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar asymptotes sleep " }
 { "l_orderkey": 2466, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17378.88, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans sl" }
-{ "l_orderkey": 2947, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10861.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly special " }
-{ "l_orderkey": 4069, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3258.54, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l packages. even, " }
-{ "l_orderkey": 5283, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1086.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-20", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits within the furio" }
-{ "l_orderkey": 230, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49964.28, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old packages ha" }
-{ "l_orderkey": 610, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18465.06, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "p quickly instead of the slyly pending foxe" }
-{ "l_orderkey": 1573, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-24", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ymptotes could u" }
 { "l_orderkey": 2823, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11947.98, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-11-24", "l_receiptdate": "1995-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "bold requests nag blithely s" }
+{ "l_orderkey": 2947, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10861.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly special " }
 { "l_orderkey": 3174, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6517.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously ironic" }
+{ "l_orderkey": 3206, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26068.32, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "encies sleep deposits--" }
 { "l_orderkey": 3298, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 29326.86, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-10", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar packages. regular deposit" }
-{ "l_orderkey": 5092, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 45619.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1996-01-01", "l_receiptdate": "1995-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s use along t" }
-{ "l_orderkey": 5444, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22809.78, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar packages haggle above th" }
-{ "l_orderkey": 325, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5430.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " theodolites. " }
+{ "l_orderkey": 3489, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20637.42, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-31", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-08-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "c deposits alongside of the pending, fu" }
 { "l_orderkey": 3623, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7603.26, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-05", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "aves. slyly special packages cajole. fu" }
 { "l_orderkey": 3653, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9775.62, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-03", "l_commitdate": "1994-05-19", "l_receiptdate": "1994-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slyly silent account" }
+{ "l_orderkey": 4069, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3258.54, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l packages. even, " }
 { "l_orderkey": 4321, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24982.14, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-03", "l_commitdate": "1994-10-08", "l_receiptdate": "1994-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly even orbits slee" }
+{ "l_orderkey": 5092, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 45619.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1996-01-01", "l_receiptdate": "1995-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s use along t" }
+{ "l_orderkey": 5283, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1086.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-20", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits within the furio" }
+{ "l_orderkey": 5286, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 41274.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-26", "l_receiptdate": "1997-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "fluffily. special, ironic deposit" }
+{ "l_orderkey": 5444, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22809.78, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar packages haggle above th" }
+{ "l_orderkey": 5670, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46705.74, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ests in place of the carefully sly depos" }
 { "l_orderkey": 5891, "l_partkey": 186, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9775.62, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-02-27", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cajole carefully " }
+{ "l_orderkey": 580, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20618.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "mong the special packag" }
+{ "l_orderkey": 612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5425.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "structions. q" }
 { "l_orderkey": 835, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30385.04, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-11", "l_receiptdate": "1996-01-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " fluffily furious pinto beans" }
+{ "l_orderkey": 1347, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24959.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ronic pinto beans. express reques" }
+{ "l_orderkey": 1666, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32555.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-30", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " breach evenly final accounts. r" }
 { "l_orderkey": 1796, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8681.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "slyly bold accounts are furiously agains" }
-{ "l_orderkey": 3296, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31470.22, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-11-26", "l_receiptdate": "1995-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss ideas are reg" }
-{ "l_orderkey": 5634, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28214.68, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ptotes mold qu" }
+{ "l_orderkey": 2023, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9766.62, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts maintain blithely alongside of the" }
 { "l_orderkey": 2790, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29299.86, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-09-27", "l_receiptdate": "1994-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ilent packages cajole. quickly ironic requ" }
+{ "l_orderkey": 3009, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41236.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal packages should haggle slyly. quickl" }
 { "l_orderkey": 3173, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2170.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-18", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fluffily above t" }
 { "l_orderkey": 3267, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35810.94, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-30", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es boost. " }
+{ "l_orderkey": 3296, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31470.22, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-11-26", "l_receiptdate": "1995-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ss ideas are reg" }
 { "l_orderkey": 3712, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14107.34, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-02-11", "l_receiptdate": "1992-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s around the furiously ironic account" }
-{ "l_orderkey": 4609, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3255.54, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nstructions. furious instructions " }
-{ "l_orderkey": 4643, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54259.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". ironic deposits cajo" }
-{ "l_orderkey": 580, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20618.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "mong the special packag" }
-{ "l_orderkey": 1666, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32555.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-30", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " breach evenly final accounts. r" }
-{ "l_orderkey": 4612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10851.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-11", "l_commitdate": "1993-11-19", "l_receiptdate": "1993-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual theodol" }
-{ "l_orderkey": 612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5425.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "structions. q" }
-{ "l_orderkey": 1347, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24959.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ronic pinto beans. express reques" }
-{ "l_orderkey": 2023, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9766.62, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts maintain blithely alongside of the" }
-{ "l_orderkey": 3009, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41236.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal packages should haggle slyly. quickl" }
 { "l_orderkey": 4389, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4340.72, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " blithely even d" }
 { "l_orderkey": 4577, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 46662.74, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "packages. " }
+{ "l_orderkey": 4609, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3255.54, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nstructions. furious instructions " }
+{ "l_orderkey": 4612, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10851.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-11", "l_commitdate": "1993-11-19", "l_receiptdate": "1993-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual theodol" }
+{ "l_orderkey": 4643, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54259.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". ironic deposits cajo" }
 { "l_orderkey": 4739, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33640.58, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely special pin" }
 { "l_orderkey": 5574, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49918.28, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "arefully express requests wake furiousl" }
-{ "l_orderkey": 390, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial excuses. bold, pending packages" }
-{ "l_orderkey": 897, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28188.68, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-07-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "tions sleep according to the special" }
-{ "l_orderkey": 1731, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-18", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ngside of the even instruct" }
-{ "l_orderkey": 1763, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 2168.36, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1996-12-04", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even pinto beans snooze fluffi" }
-{ "l_orderkey": 1925, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-23", "l_receiptdate": "1992-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "usual pinto" }
-{ "l_orderkey": 3303, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-04-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly regular pi" }
-{ "l_orderkey": 4064, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1997-01-05", "l_receiptdate": "1996-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "alongside of the f" }
-{ "l_orderkey": 4583, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests haggle after the furiously " }
-{ "l_orderkey": 4705, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " above the furiously ev" }
-{ "l_orderkey": 5121, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even courts are blithely ironically " }
-{ "l_orderkey": 5380, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43367.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-27", "l_receiptdate": "1998-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ar asymptotes. blithely r" }
-{ "l_orderkey": 5472, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egrate carefully dependencies. " }
+{ "l_orderkey": 5634, "l_partkey": 185, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28214.68, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ptotes mold qu" }
+{ "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
 { "l_orderkey": 194, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-05-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular theodolites. regular, iron" }
+{ "l_orderkey": 322, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10841.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits grow slyly according to th" }
+{ "l_orderkey": 326, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-27", "l_receiptdate": "1995-08-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily furiously unusual accounts. " }
+{ "l_orderkey": 390, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial excuses. bold, pending packages" }
 { "l_orderkey": 484, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-03-27", "l_receiptdate": "1997-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uctions wake. final, silent requests haggle" }
+{ "l_orderkey": 897, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28188.68, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-07-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "tions sleep according to the special" }
+{ "l_orderkey": 1024, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14094.34, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e slyly around the slyly special instructi" }
 { "l_orderkey": 1092, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 52040.64, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "unusual accounts. fluffi" }
 { "l_orderkey": 1286, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-11", "l_receiptdate": "1993-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironic pinto beans cajole furiously s" }
+{ "l_orderkey": 1731, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-18", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ngside of the even instruct" }
+{ "l_orderkey": 1763, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 2168.36, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1996-12-04", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even pinto beans snooze fluffi" }
+{ "l_orderkey": 1891, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19515.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-02-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " foxes above the carefu" }
+{ "l_orderkey": 1923, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the ideas: slyly pendin" }
+{ "l_orderkey": 1925, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-23", "l_receiptdate": "1992-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "usual pinto" }
 { "l_orderkey": 2115, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46619.74, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully pending requests alongs" }
 { "l_orderkey": 2116, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11925.98, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-09-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pinto beans. final, final sauternes play " }
+{ "l_orderkey": 2118, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4336.72, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-25", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "theodolites affix according " }
+{ "l_orderkey": 2273, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36862.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " furiously carefully bold de" }
 { "l_orderkey": 2533, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21683.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-04-30", "l_receiptdate": "1997-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thless excuses are b" }
-{ "l_orderkey": 3105, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11925.98, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-07", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly bold depths caj" }
-{ "l_orderkey": 3394, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15178.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-07-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "arefully regular do" }
-{ "l_orderkey": 3525, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-08", "l_receiptdate": "1996-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " nag according " }
-{ "l_orderkey": 322, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10841.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits grow slyly according to th" }
-{ "l_orderkey": 1923, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the ideas: slyly pendin" }
 { "l_orderkey": 2695, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22767.78, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y regular pinto beans. evenly regular packa" }
 { "l_orderkey": 2790, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20599.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-10-03", "l_receiptdate": "1994-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uffily even excuses. furiously thin" }
 { "l_orderkey": 2886, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41198.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-01-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old requests along the fur" }
+{ "l_orderkey": 3105, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11925.98, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-07", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly bold depths caj" }
+{ "l_orderkey": 3303, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-04-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly regular pi" }
+{ "l_orderkey": 3394, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15178.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-07-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "arefully regular do" }
 { "l_orderkey": 3397, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-03", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " regular packag" }
-{ "l_orderkey": 4069, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ages. carefully regular " }
-{ "l_orderkey": 4992, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45535.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-19", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "foxes about the quickly final platele" }
-{ "l_orderkey": 5474, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41198.84, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-07-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly beneath " }
-{ "l_orderkey": 5543, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously. slyly" }
-{ "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
-{ "l_orderkey": 326, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27104.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-27", "l_receiptdate": "1995-08-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily furiously unusual accounts. " }
-{ "l_orderkey": 1024, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14094.34, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e slyly around the slyly special instructi" }
-{ "l_orderkey": 1891, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19515.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-02-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " foxes above the carefu" }
-{ "l_orderkey": 2118, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4336.72, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-25", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "theodolites affix according " }
-{ "l_orderkey": 2273, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36862.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " furiously carefully bold de" }
+{ "l_orderkey": 3525, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 30357.04, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-08", "l_receiptdate": "1996-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " nag according " }
 { "l_orderkey": 3655, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5420.9, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-17", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "riously bold pinto be" }
 { "l_orderkey": 3810, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53124.82, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "cajole. fur" }
 { "l_orderkey": 3904, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20599.42, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " excuses sleep slyly according to th" }
+{ "l_orderkey": 4064, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49872.28, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1997-01-05", "l_receiptdate": "1996-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "alongside of the f" }
+{ "l_orderkey": 4069, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 54209.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ages. carefully regular " }
+{ "l_orderkey": 4583, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39030.48, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests haggle after the furiously " }
+{ "l_orderkey": 4705, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " above the furiously ev" }
 { "l_orderkey": 4801, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uests hinder blithely against the instr" }
+{ "l_orderkey": 4992, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45535.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-19", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "foxes about the quickly final platele" }
+{ "l_orderkey": 5121, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24936.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even courts are blithely ironically " }
+{ "l_orderkey": 5380, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43367.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-27", "l_receiptdate": "1998-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ar asymptotes. blithely r" }
+{ "l_orderkey": 5472, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 40114.66, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egrate carefully dependencies. " }
+{ "l_orderkey": 5474, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41198.84, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-07-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly beneath " }
+{ "l_orderkey": 5543, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1084.18, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously. slyly" }
+{ "l_orderkey": 7, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12998.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss pinto beans wake against th" }
+{ "l_orderkey": 102, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bits. ironic accoun" }
+{ "l_orderkey": 260, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28162.68, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-02-06", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld theodolites boost fl" }
+{ "l_orderkey": 359, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24913.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1995-03-11", "l_receiptdate": "1995-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic courts snooze quickly furiously final fo" }
 { "l_orderkey": 515, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11914.98, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-10-02", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly pending accounts haggle blithel" }
-{ "l_orderkey": 1570, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its. slyly regular sentiments" }
-{ "l_orderkey": 1605, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ole carefully car" }
-{ "l_orderkey": 3143, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sly unusual theodolites. slyly ev" }
-{ "l_orderkey": 3942, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6499.08, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ep ruthlessly carefully final accounts: s" }
-{ "l_orderkey": 5408, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8665.44, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-09-06", "l_receiptdate": "1992-11-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "thely regular hocke" }
-{ "l_orderkey": 5792, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34661.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are slyly against the ev" }
 { "l_orderkey": 774, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53075.82, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1996-01-07", "l_receiptdate": "1995-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ess accounts are carefully " }
+{ "l_orderkey": 1570, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its. slyly regular sentiments" }
 { "l_orderkey": 1571, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6499.08, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1993-02-13", "l_receiptdate": "1993-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " special, ironic depo" }
+{ "l_orderkey": 1602, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4332.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-31", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y. even excuses" }
+{ "l_orderkey": 1605, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ole carefully car" }
+{ "l_orderkey": 2150, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37911.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending dependen" }
 { "l_orderkey": 2438, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 49826.28, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic requests cajole f" }
 { "l_orderkey": 2533, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40077.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " haggle carefully " }
 { "l_orderkey": 2721, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53075.82, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-14", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ounts poach carefu" }
+{ "l_orderkey": 3143, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sly unusual theodolites. slyly ev" }
 { "l_orderkey": 3143, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23829.96, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "beans. fluf" }
+{ "l_orderkey": 3202, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32495.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-03-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven platelets. furiously final" }
+{ "l_orderkey": 3299, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly even request" }
 { "l_orderkey": 3749, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15164.52, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "press instruc" }
+{ "l_orderkey": 3942, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6499.08, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ep ruthlessly carefully final accounts: s" }
 { "l_orderkey": 4070, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2166.36, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ptotes affix" }
 { "l_orderkey": 4710, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cross the blithely bold packages. silen" }
 { "l_orderkey": 4834, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29245.86, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-10-27", "l_receiptdate": "1997-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "es nag blithe" }
@@ -553,5453 +559,5447 @@
 { "l_orderkey": 4998, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16247.7, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "heodolites sleep quickly." }
 { "l_orderkey": 5122, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30329.04, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-03-29", "l_receiptdate": "1996-04-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "g the busily ironic accounts boos" }
 { "l_orderkey": 5191, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7582.26, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-30", "l_receiptdate": "1995-03-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "eposits. express" }
-{ "l_orderkey": 7, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12998.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss pinto beans wake against th" }
-{ "l_orderkey": 260, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28162.68, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-02-06", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld theodolites boost fl" }
-{ "l_orderkey": 359, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24913.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1995-03-11", "l_receiptdate": "1995-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic courts snooze quickly furiously final fo" }
-{ "l_orderkey": 1602, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4332.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-31", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y. even excuses" }
+{ "l_orderkey": 5408, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8665.44, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-09-06", "l_receiptdate": "1992-11-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "thely regular hocke" }
 { "l_orderkey": 5475, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10831.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-22", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ding to the deposits wake fina" }
 { "l_orderkey": 5734, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31412.22, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions cajole final, express " }
-{ "l_orderkey": 102, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 27079.5, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-31", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bits. ironic accoun" }
-{ "l_orderkey": 2150, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37911.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending dependen" }
-{ "l_orderkey": 3202, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32495.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-03-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven platelets. furiously final" }
-{ "l_orderkey": 3299, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43327.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly even request" }
+{ "l_orderkey": 5792, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34661.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are slyly against the ev" }
 { "l_orderkey": 384, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11903.98, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-02", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ash carefully" }
 { "l_orderkey": 547, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3246.54, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-04", "l_commitdate": "1996-08-01", "l_receiptdate": "1996-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "pinto beans. ironi" }
-{ "l_orderkey": 1122, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ptotes. quickl" }
-{ "l_orderkey": 5634, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-10", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final ideas. deposits sleep. reg" }
 { "l_orderkey": 737, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12986.16, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits after the slyly bold du" }
+{ "l_orderkey": 1057, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21643.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s wake bol" }
+{ "l_orderkey": 1122, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ptotes. quickl" }
+{ "l_orderkey": 1510, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8657.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely express" }
+{ "l_orderkey": 1954, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1082.18, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "te. furiously final deposits hag" }
+{ "l_orderkey": 2022, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17314.88, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ages wake slyly care" }
+{ "l_orderkey": 2084, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45451.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y fluffily even foxes. " }
+{ "l_orderkey": 2401, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42205.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-10-21", "l_receiptdate": "1997-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ould affix " }
+{ "l_orderkey": 3457, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-12", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully final excuses wake" }
+{ "l_orderkey": 3713, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-08-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "the regular dugouts wake furiously sil" }
+{ "l_orderkey": 3716, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27054.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-23", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fully unusual accounts. carefu" }
+{ "l_orderkey": 3719, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12986.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-05-04", "l_receiptdate": "1997-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "grate according to the " }
+{ "l_orderkey": 3810, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11903.98, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-11", "l_receiptdate": "1993-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the pending pinto beans. expr" }
 { "l_orderkey": 3811, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 24890.14, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nstructions sleep quickly. slyly final " }
 { "l_orderkey": 4035, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14068.34, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-07-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s. furiously even courts wake slyly" }
 { "l_orderkey": 4706, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40040.66, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-20", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "kly final deposits c" }
-{ "l_orderkey": 5505, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35711.94, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-11-11", "l_receiptdate": "1998-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely unusual excuses integrat" }
-{ "l_orderkey": 5664, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9739.62, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-04", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly. express ideas agai" }
-{ "l_orderkey": 1057, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21643.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s wake bol" }
-{ "l_orderkey": 2022, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17314.88, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ages wake slyly care" }
-{ "l_orderkey": 2084, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45451.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y fluffily even foxes. " }
-{ "l_orderkey": 3457, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-12", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully final excuses wake" }
-{ "l_orderkey": 3716, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 27054.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-23", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fully unusual accounts. carefu" }
 { "l_orderkey": 5380, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15150.52, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-18", "l_commitdate": "1997-12-03", "l_receiptdate": "1998-01-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "final platelets." }
-{ "l_orderkey": 1510, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8657.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely express" }
-{ "l_orderkey": 1954, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1082.18, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "te. furiously final deposits hag" }
-{ "l_orderkey": 2401, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42205.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-10-21", "l_receiptdate": "1997-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ould affix " }
-{ "l_orderkey": 3713, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-08-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "the regular dugouts wake furiously sil" }
-{ "l_orderkey": 3719, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12986.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-05-04", "l_receiptdate": "1997-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "grate according to the " }
-{ "l_orderkey": 3810, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11903.98, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-11", "l_receiptdate": "1993-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the pending pinto beans. expr" }
+{ "l_orderkey": 5505, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35711.94, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-11-11", "l_receiptdate": "1998-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely unusual excuses integrat" }
+{ "l_orderkey": 5634, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 31383.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-10", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final ideas. deposits sleep. reg" }
+{ "l_orderkey": 5664, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9739.62, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-04", "l_commitdate": "1998-10-15", "l_receiptdate": "1998-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly. express ideas agai" }
 { "l_orderkey": 5824, "l_partkey": 182, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 45451.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts sleep. carefully regular accounts h" }
 { "l_orderkey": 770, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "osits. foxes cajole " }
-{ "l_orderkey": 2566, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-12-22", "l_receiptdate": "1992-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ously ironic accounts" }
-{ "l_orderkey": 3013, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18380.06, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-05-02", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fully unusual account" }
-{ "l_orderkey": 4293, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits should boost along the " }
-{ "l_orderkey": 999, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 40003.66, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1994-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckly slyly unusual packages: packages hagg" }
-{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31354.22, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " haggle: closely even asymptot" }
-{ "l_orderkey": 1890, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 17298.88, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ged pinto beans. regular, regular id" }
-{ "l_orderkey": 2209, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-09-02", "l_receiptdate": "1992-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly around the final packages. deposits ca" }
-{ "l_orderkey": 2656, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10811.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-28", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s nag regularly about the deposits. slyly" }
-{ "l_orderkey": 3041, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "posits dazzle special p" }
-{ "l_orderkey": 3424, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-03", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-11-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "bits boost closely slyly p" }
-{ "l_orderkey": 3872, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30273.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "t after the carefully ironic excuses. f" }
 { "l_orderkey": 807, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 51896.64, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "kly across the f" }
 { "l_orderkey": 998, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7568.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits. even asym" }
-{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11892.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "furiously unusual packages play quickly " }
-{ "l_orderkey": 2821, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4324.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-15", "l_commitdate": "1993-10-02", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nding foxes." }
-{ "l_orderkey": 5540, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss dolphins haggle " }
+{ "l_orderkey": 999, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 40003.66, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1994-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckly slyly unusual packages: packages hagg" }
 { "l_orderkey": 1088, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-07-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully ironic packages. r" }
 { "l_orderkey": 1095, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 40003.66, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-04", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-10-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". quickly even dolphins sle" }
 { "l_orderkey": 1349, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-07", "l_commitdate": "1998-01-14", "l_receiptdate": "1998-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " express inst" }
+{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31354.22, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " haggle: closely even asymptot" }
+{ "l_orderkey": 1382, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11892.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "furiously unusual packages play quickly " }
+{ "l_orderkey": 1890, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 17298.88, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ged pinto beans. regular, regular id" }
+{ "l_orderkey": 2209, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-09-02", "l_receiptdate": "1992-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly around the final packages. deposits ca" }
+{ "l_orderkey": 2566, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-12-22", "l_receiptdate": "1992-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ously ironic accounts" }
+{ "l_orderkey": 2656, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10811.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-28", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s nag regularly about the deposits. slyly" }
+{ "l_orderkey": 2821, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4324.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-15", "l_commitdate": "1993-10-02", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nding foxes." }
 { "l_orderkey": 2854, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49734.28, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". furiously regular deposits across th" }
+{ "l_orderkey": 3013, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18380.06, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-05-02", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fully unusual account" }
+{ "l_orderkey": 3041, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "posits dazzle special p" }
 { "l_orderkey": 3232, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3243.54, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-12-11", "l_receiptdate": "1992-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ily blithely ironic acco" }
+{ "l_orderkey": 3424, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42166.02, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-03", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-11-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "bits boost closely slyly p" }
 { "l_orderkey": 3653, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18380.06, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-07-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gle slyly regular" }
+{ "l_orderkey": 3872, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30273.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "t after the carefully ironic excuses. f" }
+{ "l_orderkey": 4293, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits should boost along the " }
 { "l_orderkey": 4417, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1081.18, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-23", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-10-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "press deposits promise stealthily amo" }
 { "l_orderkey": 4675, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5405.9, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lent pinto beans" }
 { "l_orderkey": 5031, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 33516.58, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ts across the even requests doze furiously" }
+{ "l_orderkey": 5540, "l_partkey": 181, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 45409.56, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss dolphins haggle " }
 { "l_orderkey": 70, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1080.18, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-03-05", "l_receiptdate": "1994-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "quickly. fluffily unusual theodolites c" }
+{ "l_orderkey": 326, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ily quickly bold ideas." }
+{ "l_orderkey": 512, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43207.2, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "quests are da" }
+{ "l_orderkey": 640, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23763.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "osits across the slyly regular theodo" }
 { "l_orderkey": 708, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-28", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " requests. even, thin ideas" }
+{ "l_orderkey": 772, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10801.8, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "o the furiously final deposits. furi" }
 { "l_orderkey": 899, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-21", "l_commitdate": "1998-05-28", "l_receiptdate": "1998-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ades impress carefully" }
 { "l_orderkey": 966, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "efully final pinto beans. quickly " }
+{ "l_orderkey": 1253, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-03", "l_commitdate": "1993-04-16", "l_receiptdate": "1993-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lar foxes sleep furiously final, final pack" }
+{ "l_orderkey": 1605, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19443.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-13", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly regular foxes wake carefully. bol" }
 { "l_orderkey": 1826, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6481.08, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-30", "l_commitdate": "1992-05-17", "l_receiptdate": "1992-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "kages. blithely silent" }
+{ "l_orderkey": 1923, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11881.98, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-09-04", "l_receiptdate": "1997-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ages wake slyly about the furiously regular" }
+{ "l_orderkey": 2084, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es against " }
 { "l_orderkey": 2881, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17282.88, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-06-27", "l_receiptdate": "1992-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "usly bold " }
 { "l_orderkey": 2949, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 41046.84, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se slyly requests. carefull" }
+{ "l_orderkey": 3431, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-26", "l_commitdate": "1993-10-13", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " sleep carefully ironically special" }
 { "l_orderkey": 3652, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25924.32, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "the final p" }
 { "l_orderkey": 3713, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20523.42, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quests cajole careful" }
-{ "l_orderkey": 4099, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 49688.28, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-29", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages nag requests." }
-{ "l_orderkey": 5318, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28084.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al, express foxes. bold requests sleep alwa" }
-{ "l_orderkey": 5382, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6481.08, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-07", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes by the sl" }
-{ "l_orderkey": 326, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ily quickly bold ideas." }
-{ "l_orderkey": 1605, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19443.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-13", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly regular foxes wake carefully. bol" }
-{ "l_orderkey": 2084, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es against " }
 { "l_orderkey": 3906, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16202.7, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "dependencies at the " }
-{ "l_orderkey": 4515, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-15", "l_receiptdate": "1992-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ns. bold r" }
-{ "l_orderkey": 4868, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8641.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly special th" }
-{ "l_orderkey": 5221, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17282.88, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-29", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ending request" }
-{ "l_orderkey": 512, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43207.2, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "quests are da" }
-{ "l_orderkey": 772, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10801.8, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "o the furiously final deposits. furi" }
-{ "l_orderkey": 1923, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11881.98, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-09-04", "l_receiptdate": "1997-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ages wake slyly about the furiously regular" }
 { "l_orderkey": 4067, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19443.24, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e the slyly final packages d" }
+{ "l_orderkey": 4099, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 49688.28, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-29", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages nag requests." }
+{ "l_orderkey": 4515, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24844.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-15", "l_receiptdate": "1992-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ns. bold r" }
 { "l_orderkey": 4642, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36726.12, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-05-11", "l_receiptdate": "1995-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "theodolites detect among the ironically sp" }
+{ "l_orderkey": 4868, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8641.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly special th" }
 { "l_orderkey": 4964, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12962.16, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-09-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully silent instructions ca" }
 { "l_orderkey": 5092, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-20", "l_commitdate": "1995-11-30", "l_receiptdate": "1996-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " deposits cajole furiously against the sly" }
-{ "l_orderkey": 640, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23763.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "osits across the slyly regular theodo" }
-{ "l_orderkey": 1253, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15122.52, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-03", "l_commitdate": "1993-04-16", "l_receiptdate": "1993-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lar foxes sleep furiously final, final pack" }
-{ "l_orderkey": 3431, "l_partkey": 180, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44287.38, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-26", "l_commitdate": "1993-10-13", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " sleep carefully ironically special" }
-{ "l_orderkey": 1410, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19425.06, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-03", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gle furiously fluffily regular requests" }
-{ "l_orderkey": 1537, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53958.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "special packages haggle slyly at the silent" }
-{ "l_orderkey": 2343, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-10-26", "l_receiptdate": "1995-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "osits. unusual theodolites boost furio" }
-{ "l_orderkey": 4193, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "uffily spe" }
-{ "l_orderkey": 4551, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28058.42, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "le. carefully dogged accounts use furiousl" }
-{ "l_orderkey": 4578, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16187.55, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular theodo" }
-{ "l_orderkey": 5922, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly regular deposits haggle quickly ins" }
+{ "l_orderkey": 5221, "l_partkey": 180, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17282.88, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-29", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ending request" }
+{ "l_orderkey": 5318, "l_partkey": 180, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28084.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al, express foxes. bold requests sleep alwa" }
+{ "l_orderkey": 5382, "l_partkey": 180, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6481.08, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-07", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes by the sl" }
 { "l_orderkey": 67, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 31295.93, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ultipliers " }
 { "l_orderkey": 384, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 41008.46, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "totes cajole blithely against the even" }
 { "l_orderkey": 898, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39929.29, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-17", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "packages sleep furiously" }
+{ "l_orderkey": 1188, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-29", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "althy packages. fluffily unusual ideas h" }
+{ "l_orderkey": 1287, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y quickly bold theodoli" }
+{ "l_orderkey": 1410, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19425.06, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-03", "l_commitdate": "1997-05-17", "l_receiptdate": "1997-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gle furiously fluffily regular requests" }
+{ "l_orderkey": 1537, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53958.5, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "special packages haggle slyly at the silent" }
+{ "l_orderkey": 2182, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39929.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ges. blithely ironic" }
+{ "l_orderkey": 2343, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-10-26", "l_receiptdate": "1995-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "osits. unusual theodolites boost furio" }
+{ "l_orderkey": 3459, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33454.27, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-10-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y regular pain" }
+{ "l_orderkey": 4066, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52879.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-02-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ial braids. furiously final deposits sl" }
+{ "l_orderkey": 4193, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "uffily spe" }
+{ "l_orderkey": 4551, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28058.42, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "le. carefully dogged accounts use furiousl" }
+{ "l_orderkey": 4578, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16187.55, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular theodo" }
+{ "l_orderkey": 4642, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s are blithely. requests wake above the fur" }
+{ "l_orderkey": 4741, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43166.8, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " fluffily slow deposits. fluffily regu" }
 { "l_orderkey": 4835, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19425.06, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-17", "l_commitdate": "1994-12-14", "l_receiptdate": "1995-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "eat furiously against the slyly " }
 { "l_orderkey": 5315, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42087.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-12-29", "l_receiptdate": "1992-12-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly alongside of the ca" }
-{ "l_orderkey": 1188, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-29", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "althy packages. fluffily unusual ideas h" }
-{ "l_orderkey": 2182, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39929.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ges. blithely ironic" }
-{ "l_orderkey": 3459, "l_partkey": 179, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33454.27, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-10-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y regular pain" }
-{ "l_orderkey": 4741, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43166.8, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " fluffily slow deposits. fluffily regu" }
-{ "l_orderkey": 1287, "l_partkey": 179, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22662.57, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y quickly bold theodoli" }
-{ "l_orderkey": 4066, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52879.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-02-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ial braids. furiously final deposits sl" }
-{ "l_orderkey": 4642, "l_partkey": 179, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 44245.97, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s are blithely. requests wake above the fur" }
+{ "l_orderkey": 5922, "l_partkey": 179, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10791.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly regular deposits haggle quickly ins" }
+{ "l_orderkey": 197, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8625.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-17", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y blithely even deposits. blithely fina" }
+{ "l_orderkey": 1059, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17250.72, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pinto " }
+{ "l_orderkey": 1120, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10781.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "dependencies. blithel" }
+{ "l_orderkey": 1123, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42048.63, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "rding to the furiously ironic requests: r" }
 { "l_orderkey": 1187, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31266.93, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1993-02-09", "l_receiptdate": "1992-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "riously express ac" }
+{ "l_orderkey": 1284, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-11", "l_commitdate": "1996-03-04", "l_receiptdate": "1996-04-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar packages. special packages ac" }
+{ "l_orderkey": 1286, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-24", "l_commitdate": "1993-08-12", "l_receiptdate": "1993-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gged accoun" }
 { "l_orderkey": 1382, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46361.31, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-02", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ress deposits. slyly ironic foxes are blit" }
 { "l_orderkey": 1413, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19407.06, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-11", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-10-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly bold packages haggle quickly acr" }
+{ "l_orderkey": 1504, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9703.53, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-12", "l_receiptdate": "1992-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly regular courts." }
+{ "l_orderkey": 1538, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14016.21, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-26", "l_commitdate": "1995-07-30", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. packages sleep f" }
+{ "l_orderkey": 1575, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15094.38, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans breach among the furiously specia" }
 { "l_orderkey": 1607, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51752.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-22", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ular forges. deposits a" }
 { "l_orderkey": 1633, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 37735.95, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-09", "l_commitdate": "1995-12-02", "l_receiptdate": "1996-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly against the dolph" }
+{ "l_orderkey": 1671, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "luffily regular deposits" }
+{ "l_orderkey": 1825, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 35579.61, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-03-01", "l_receiptdate": "1993-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ne" }
+{ "l_orderkey": 1923, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-08", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "aggle carefully. furiously permanent" }
 { "l_orderkey": 1926, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10781.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-06-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "usly bold accounts. express accounts" }
+{ "l_orderkey": 2181, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4312.68, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tes. slyly silent packages use along th" }
 { "l_orderkey": 2209, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7547.19, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " quickly regular pack" }
+{ "l_orderkey": 2306, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37735.95, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-08-30", "l_receiptdate": "1995-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "raids along the furiously unusual asympto" }
 { "l_orderkey": 2661, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33423.27, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e ironicall" }
+{ "l_orderkey": 3173, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-06", "l_commitdate": "1996-09-17", "l_receiptdate": "1996-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "express depo" }
+{ "l_orderkey": 3235, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-16", "l_commitdate": "1996-01-05", "l_receiptdate": "1996-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ldly ironic pinto beans" }
 { "l_orderkey": 3393, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39892.29, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ss the slyly ironic pinto beans. ironic," }
 { "l_orderkey": 3521, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 40970.46, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1992-12-10", "l_receiptdate": "1993-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ges hang q" }
 { "l_orderkey": 3809, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 46361.31, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-06", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "yly ironic decoys; regular, iron" }
-{ "l_orderkey": 4646, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28032.42, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-08-25", "l_receiptdate": "1996-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ix according to the slyly spe" }
-{ "l_orderkey": 4678, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43126.8, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-10-27", "l_receiptdate": "1998-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". final, unusual requests sleep thinl" }
-{ "l_orderkey": 5472, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 48517.65, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " idle packages. furi" }
-{ "l_orderkey": 5792, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36657.78, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-06-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "requests are against t" }
-{ "l_orderkey": 1123, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42048.63, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "rding to the furiously ironic requests: r" }
-{ "l_orderkey": 1286, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-24", "l_commitdate": "1993-08-12", "l_receiptdate": "1993-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gged accoun" }
-{ "l_orderkey": 2306, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37735.95, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-08-30", "l_receiptdate": "1995-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "raids along the furiously unusual asympto" }
-{ "l_orderkey": 4196, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49595.82, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-06-28", "l_receiptdate": "1998-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "according to t" }
-{ "l_orderkey": 4580, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1993-12-17", "l_receiptdate": "1994-02-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "o beans. f" }
-{ "l_orderkey": 5095, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45283.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ccounts. packages could have t" }
-{ "l_orderkey": 1120, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10781.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "dependencies. blithel" }
-{ "l_orderkey": 1538, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14016.21, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-26", "l_commitdate": "1995-07-30", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. packages sleep f" }
-{ "l_orderkey": 1671, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "luffily regular deposits" }
-{ "l_orderkey": 1825, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 35579.61, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-03-01", "l_receiptdate": "1993-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ne" }
-{ "l_orderkey": 2181, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4312.68, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tes. slyly silent packages use along th" }
 { "l_orderkey": 3909, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32345.1, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-10-14", "l_receiptdate": "1998-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly even deposits across the ironic notorni" }
 { "l_orderkey": 3940, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35579.61, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-19", "l_receiptdate": "1996-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly ironic packages about the pending accou" }
-{ "l_orderkey": 4131, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34501.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-02", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " furiously regular asymptotes nod sly" }
-{ "l_orderkey": 197, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8625.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-17", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y blithely even deposits. blithely fina" }
-{ "l_orderkey": 1059, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17250.72, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pinto " }
-{ "l_orderkey": 1284, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 52830.33, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-11", "l_commitdate": "1996-03-04", "l_receiptdate": "1996-04-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar packages. special packages ac" }
-{ "l_orderkey": 1504, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9703.53, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-12", "l_receiptdate": "1992-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly regular courts." }
-{ "l_orderkey": 1575, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 15094.38, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans breach among the furiously specia" }
-{ "l_orderkey": 1923, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-08", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "aggle carefully. furiously permanent" }
-{ "l_orderkey": 3173, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-06", "l_commitdate": "1996-09-17", "l_receiptdate": "1996-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "express depo" }
-{ "l_orderkey": 3235, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24797.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-16", "l_commitdate": "1996-01-05", "l_receiptdate": "1996-03-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ldly ironic pinto beans" }
 { "l_orderkey": 4130, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47439.48, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-14", "l_commitdate": "1996-04-15", "l_receiptdate": "1996-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eaves haggle qui" }
+{ "l_orderkey": 4131, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 34501.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-02", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " furiously regular asymptotes nod sly" }
+{ "l_orderkey": 4196, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49595.82, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-06-28", "l_receiptdate": "1998-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "according to t" }
 { "l_orderkey": 4579, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 36657.78, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-26", "l_commitdate": "1996-02-22", "l_receiptdate": "1996-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "hely. carefully blithe dependen" }
+{ "l_orderkey": 4580, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5390.85, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1993-12-17", "l_receiptdate": "1994-02-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "o beans. f" }
+{ "l_orderkey": 4646, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28032.42, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-08-25", "l_receiptdate": "1996-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ix according to the slyly spe" }
+{ "l_orderkey": 4678, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 43126.8, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-10-27", "l_receiptdate": "1998-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". final, unusual requests sleep thinl" }
 { "l_orderkey": 5092, "l_partkey": 178, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11859.87, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-12-27", "l_receiptdate": "1995-12-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly against the slyly silen" }
+{ "l_orderkey": 5095, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45283.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ccounts. packages could have t" }
 { "l_orderkey": 5443, "l_partkey": 178, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15094.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-27", "l_commitdate": "1996-11-11", "l_receiptdate": "1996-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s after the regular, regular deposits hag" }
-{ "l_orderkey": 1093, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39855.29, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-06", "l_commitdate": "1997-10-08", "l_receiptdate": "1997-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le furiously across the carefully sp" }
-{ "l_orderkey": 1441, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5385.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he quickly enticing pac" }
-{ "l_orderkey": 1954, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 14003.21, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic instructions cajole" }
-{ "l_orderkey": 2852, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6463.02, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-02", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts above the furiously un" }
-{ "l_orderkey": 3296, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17234.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-12-27", "l_receiptdate": "1995-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "kages cajole carefully " }
-{ "l_orderkey": 3622, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9694.53, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1996-02-09", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "arefully. furiously regular ideas n" }
-{ "l_orderkey": 4577, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46318.31, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-24", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly accounts. carefully " }
-{ "l_orderkey": 4644, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4308.68, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests? pendi" }
-{ "l_orderkey": 774, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s according to the deposits unwind ca" }
-{ "l_orderkey": 2405, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24774.91, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "t wake blithely blithely regular idea" }
-{ "l_orderkey": 4387, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sleep slyly. blithely sl" }
-{ "l_orderkey": 5923, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29083.59, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "arefully i" }
-{ "l_orderkey": 896, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar, pending packages. deposits are q" }
-{ "l_orderkey": 1632, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50626.99, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts. blithely regular " }
-{ "l_orderkey": 3141, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34469.44, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-12-18", "l_receiptdate": "1995-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "oxes are quickly about t" }
-{ "l_orderkey": 3169, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49549.82, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites are fl" }
-{ "l_orderkey": 3680, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "packages. quickly fluff" }
-{ "l_orderkey": 3713, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20466.23, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-25", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tructions serve blithely around the furi" }
-{ "l_orderkey": 4871, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15080.38, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "inst the never ironic " }
+{ "l_orderkey": 5472, "l_partkey": 178, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 48517.65, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " idle packages. furi" }
+{ "l_orderkey": 5792, "l_partkey": 178, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36657.78, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-06-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "requests are against t" }
 { "l_orderkey": 133, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12926.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1998-01-15", "l_receiptdate": "1997-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts cajole fluffily quickly i" }
 { "l_orderkey": 229, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3231.51, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "posits. furiously regular theodol" }
+{ "l_orderkey": 774, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s according to the deposits unwind ca" }
+{ "l_orderkey": 896, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47395.48, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar, pending packages. deposits are q" }
+{ "l_orderkey": 1093, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39855.29, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-06", "l_commitdate": "1997-10-08", "l_receiptdate": "1997-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le furiously across the carefully sp" }
+{ "l_orderkey": 1441, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5385.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he quickly enticing pac" }
+{ "l_orderkey": 1632, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50626.99, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts. blithely regular " }
+{ "l_orderkey": 1954, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 14003.21, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic instructions cajole" }
 { "l_orderkey": 1956, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8617.36, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1992-11-24", "l_receiptdate": "1993-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully about the ironic, ironic de" }
+{ "l_orderkey": 2405, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24774.91, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "t wake blithely blithely regular idea" }
 { "l_orderkey": 2754, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20466.23, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-05-06", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "latelets hag" }
 { "l_orderkey": 2788, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17234.72, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " requests wake carefully. carefully si" }
+{ "l_orderkey": 2852, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6463.02, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-02", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts above the furiously un" }
+{ "l_orderkey": 3141, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34469.44, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-12-18", "l_receiptdate": "1995-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "oxes are quickly about t" }
+{ "l_orderkey": 3169, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49549.82, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites are fl" }
+{ "l_orderkey": 3296, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17234.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-12-27", "l_receiptdate": "1995-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "kages cajole carefully " }
+{ "l_orderkey": 3622, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9694.53, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1996-02-09", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "arefully. furiously regular ideas n" }
+{ "l_orderkey": 3680, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "packages. quickly fluff" }
+{ "l_orderkey": 3713, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20466.23, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-25", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tructions serve blithely around the furi" }
+{ "l_orderkey": 4387, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 51704.16, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sleep slyly. blithely sl" }
 { "l_orderkey": 4514, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 29083.59, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". slyly sile" }
 { "l_orderkey": 4548, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23697.74, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s. furiously ironic theodolites c" }
+{ "l_orderkey": 4577, "l_partkey": 177, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46318.31, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-24", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly accounts. carefully " }
+{ "l_orderkey": 4644, "l_partkey": 177, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4308.68, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests? pendi" }
 { "l_orderkey": 4709, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26929.25, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-03-03", "l_receiptdate": "1996-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inst the ironic, regul" }
+{ "l_orderkey": 4871, "l_partkey": 177, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15080.38, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "inst the never ironic " }
 { "l_orderkey": 5382, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15080.38, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " brave platelets. ev" }
 { "l_orderkey": 5504, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7540.19, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-03-15", "l_receiptdate": "1993-05-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "packages detect furiously express reques" }
-{ "l_orderkey": 769, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38742.12, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-10-15", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "es. furiously iro" }
-{ "l_orderkey": 1826, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 15066.38, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously bold pinto beans are carefully ag" }
-{ "l_orderkey": 1958, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31208.93, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "d pinto beans" }
-{ "l_orderkey": 2789, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35513.61, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "deposits. ironic " }
-{ "l_orderkey": 2981, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8609.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-21", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ng to the f" }
-{ "l_orderkey": 3206, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1076.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-16", "l_receiptdate": "1996-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y unusual foxes cajole ab" }
-{ "l_orderkey": 3907, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 51656.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nt asymptotes lose across th" }
-{ "l_orderkey": 5184, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27980.42, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-12-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " packages are" }
-{ "l_orderkey": 5602, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9685.53, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-09-14", "l_receiptdate": "1997-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar foxes; quickly ironic ac" }
+{ "l_orderkey": 5923, "l_partkey": 177, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29083.59, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "arefully i" }
 { "l_orderkey": 38, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47351.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-11-17", "l_receiptdate": "1996-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s. blithely unusual theodolites am" }
+{ "l_orderkey": 68, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " requests are unusual, regular pinto " }
 { "l_orderkey": 422, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " ideas. qu" }
 { "l_orderkey": 453, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " furiously f" }
-{ "l_orderkey": 2020, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43046.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ently across the" }
-{ "l_orderkey": 4803, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50579.99, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly final excuses. slyly express requ" }
 { "l_orderkey": 675, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 36589.78, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-10-07", "l_receiptdate": "1997-11-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final accounts unwind around the " }
+{ "l_orderkey": 769, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38742.12, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-10-15", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "es. furiously iro" }
 { "l_orderkey": 800, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 27980.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "bove the pending requests." }
-{ "l_orderkey": 2306, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 20447.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tainments nag furiously carefull" }
-{ "l_orderkey": 3109, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46275.31, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding to the foxes. " }
-{ "l_orderkey": 3841, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3228.51, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-24", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "foxes integrate " }
-{ "l_orderkey": 4800, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-28", "l_receiptdate": "1992-02-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s sleep fluffily. furiou" }
-{ "l_orderkey": 5280, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1998-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "efully carefully pen" }
-{ "l_orderkey": 5924, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-12-11", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions cajole carefully along the " }
-{ "l_orderkey": 68, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " requests are unusual, regular pinto " }
 { "l_orderkey": 1253, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24751.91, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the slyly silent re" }
+{ "l_orderkey": 1826, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 15066.38, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously bold pinto beans are carefully ag" }
+{ "l_orderkey": 1958, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 31208.93, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "d pinto beans" }
+{ "l_orderkey": 2020, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43046.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-09-14", "l_receiptdate": "1993-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ently across the" }
+{ "l_orderkey": 2306, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 20447.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tainments nag furiously carefull" }
+{ "l_orderkey": 2789, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 35513.61, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "deposits. ironic " }
+{ "l_orderkey": 2981, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8609.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-21", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ng to the f" }
+{ "l_orderkey": 3109, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46275.31, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding to the foxes. " }
+{ "l_orderkey": 3206, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1076.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-16", "l_receiptdate": "1996-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y unusual foxes cajole ab" }
 { "l_orderkey": 3365, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52732.33, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1995-01-01", "l_receiptdate": "1995-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lyly unusual asymptotes. final" }
 { "l_orderkey": 3590, "l_partkey": 176, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10761.7, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "t the quickly ironic" }
 { "l_orderkey": 3813, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39818.29, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-13", "l_commitdate": "1998-09-19", "l_receiptdate": "1998-10-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ravely special packages haggle p" }
+{ "l_orderkey": 3841, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3228.51, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-24", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "foxes integrate " }
+{ "l_orderkey": 3907, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 51656.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nt asymptotes lose across th" }
+{ "l_orderkey": 4800, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-28", "l_receiptdate": "1992-02-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s sleep fluffily. furiou" }
+{ "l_orderkey": 4803, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50579.99, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly final excuses. slyly express requ" }
+{ "l_orderkey": 5184, "l_partkey": 176, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27980.42, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-11", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-12-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " packages are" }
+{ "l_orderkey": 5280, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49503.82, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1998-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "efully carefully pen" }
+{ "l_orderkey": 5602, "l_partkey": 176, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9685.53, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-09-14", "l_receiptdate": "1997-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar foxes; quickly ironic ac" }
+{ "l_orderkey": 5924, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40894.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-12-11", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions cajole carefully along the " }
 { "l_orderkey": 5987, "l_partkey": 176, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21523.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-09-17", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ing excuses nag quickly always bold" }
 { "l_orderkey": 227, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25804.08, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uses across the blithe dependencies cajol" }
 { "l_orderkey": 416, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26879.25, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-12-03", "l_receiptdate": "1993-10-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ses boost after the bold requests." }
 { "l_orderkey": 738, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32255.1, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-05-29", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ecial instructions haggle blithely regula" }
-{ "l_orderkey": 1604, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 16127.55, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-10", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ending realms along the special, p" }
-{ "l_orderkey": 2528, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37630.95, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ", even excuses. even," }
-{ "l_orderkey": 2885, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial deposits use bold" }
-{ "l_orderkey": 3175, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47307.48, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "are carefully furiously ironic accounts. e" }
-{ "l_orderkey": 3200, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 26879.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-08", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-03-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly regular hockey players! pinto beans " }
-{ "l_orderkey": 3622, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-02-22", "l_receiptdate": "1996-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "are careful" }
-{ "l_orderkey": 4261, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3225.51, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly even deposits eat blithely alo" }
-{ "l_orderkey": 5859, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53758.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly regular deposits use. ironic" }
 { "l_orderkey": 929, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 47307.48, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. excuses cajole. carefully regu" }
+{ "l_orderkey": 960, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34405.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-19", "l_commitdate": "1994-12-17", "l_receiptdate": "1995-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "around the blithe, even pl" }
 { "l_orderkey": 993, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35480.61, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1995-11-28", "l_receiptdate": "1996-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the deposits affix agains" }
+{ "l_orderkey": 1127, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7526.19, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-05", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " idly pending pains " }
 { "l_orderkey": 1280, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5375.85, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1993-02-11", "l_receiptdate": "1993-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "beans haggle. quickly bold instructions h" }
 { "l_orderkey": 1601, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53758.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-23", "l_receiptdate": "1995-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas doubt" }
+{ "l_orderkey": 1604, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 16127.55, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-10", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ending realms along the special, p" }
 { "l_orderkey": 2497, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 30104.76, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hely bold ideas. unusual instructions ac" }
+{ "l_orderkey": 2528, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37630.95, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ", even excuses. even," }
 { "l_orderkey": 2593, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1075.17, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-23", "l_commitdate": "1993-10-25", "l_receiptdate": "1993-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " accounts wake slyly " }
-{ "l_orderkey": 2784, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43006.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas nag furiously never unusual " }
-{ "l_orderkey": 4579, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15052.38, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-01-08", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding theodolites. fluffil" }
-{ "l_orderkey": 5634, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23653.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "silently unusual foxes above the blithely" }
-{ "l_orderkey": 5956, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-06", "l_commitdate": "1998-06-29", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lyly express theodol" }
-{ "l_orderkey": 960, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34405.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-19", "l_commitdate": "1994-12-17", "l_receiptdate": "1995-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "around the blithe, even pl" }
-{ "l_orderkey": 1127, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7526.19, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-05", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " idly pending pains " }
-{ "l_orderkey": 2915, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30104.76, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "yly special " }
-{ "l_orderkey": 4485, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al accounts according to the slyly r" }
-{ "l_orderkey": 5762, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6451.02, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-05-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ironic dependencies doze carefu" }
 { "l_orderkey": 2626, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2150.34, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-19", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uffy accounts haggle furiously above" }
+{ "l_orderkey": 2784, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43006.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas nag furiously never unusual " }
+{ "l_orderkey": 2885, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "cial deposits use bold" }
+{ "l_orderkey": 2915, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 30104.76, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "yly special " }
+{ "l_orderkey": 3175, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 47307.48, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "are carefully furiously ironic accounts. e" }
+{ "l_orderkey": 3200, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 26879.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-08", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-03-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly regular hockey players! pinto beans " }
 { "l_orderkey": 3554, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 34405.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". blithely ironic t" }
 { "l_orderkey": 3557, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44081.97, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-30", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas breach c" }
+{ "l_orderkey": 3622, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-02-22", "l_receiptdate": "1996-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "are careful" }
+{ "l_orderkey": 4261, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3225.51, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly even deposits eat blithely alo" }
 { "l_orderkey": 4294, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. blithely r" }
+{ "l_orderkey": 4485, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 46232.31, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al accounts according to the slyly r" }
+{ "l_orderkey": 4579, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 15052.38, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-01-08", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding theodolites. fluffil" }
 { "l_orderkey": 4610, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 15052.38, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " ironic frays. dependencies detect blithel" }
+{ "l_orderkey": 5634, "l_partkey": 175, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23653.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "silently unusual foxes above the blithely" }
+{ "l_orderkey": 5762, "l_partkey": 175, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6451.02, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-05-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ironic dependencies doze carefu" }
+{ "l_orderkey": 5859, "l_partkey": 175, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53758.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly regular deposits use. ironic" }
+{ "l_orderkey": 5956, "l_partkey": 175, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50532.99, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-06", "l_commitdate": "1998-06-29", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lyly express theodol" }
+{ "l_orderkey": 66, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-01", "l_receiptdate": "1994-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular de" }
+{ "l_orderkey": 67, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5370.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-20", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y unusual packages thrash pinto " }
+{ "l_orderkey": 261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30076.76, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-24", "l_commitdate": "1993-08-20", "l_receiptdate": "1993-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ironic packages nag slyly. carefully fin" }
+{ "l_orderkey": 289, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26854.25, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "out the quickly bold theodol" }
+{ "l_orderkey": 580, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ose alongside of the sl" }
+{ "l_orderkey": 775, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22557.57, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " quickly sile" }
+{ "l_orderkey": 871, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4296.68, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-01-20", "l_receiptdate": "1996-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l, regular dependencies w" }
+{ "l_orderkey": 1287, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-09-12", "l_receiptdate": "1994-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s wake unusual grou" }
+{ "l_orderkey": 1606, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully sil" }
+{ "l_orderkey": 1767, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25780.08, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-16", "l_commitdate": "1995-04-29", "l_receiptdate": "1995-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "luffy theodolites need to detect furi" }
+{ "l_orderkey": 1857, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular, regular inst" }
+{ "l_orderkey": 1991, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6445.02, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hes nag slyly" }
+{ "l_orderkey": 2054, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15038.38, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uickly final" }
+{ "l_orderkey": 2213, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 41892.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "the blithely " }
 { "l_orderkey": 2305, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3222.51, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-24", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-03-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "kages haggle quickly across the blithely " }
 { "l_orderkey": 2535, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26854.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ions believe ab" }
 { "l_orderkey": 2820, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24705.91, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-10", "l_commitdate": "1994-08-08", "l_receiptdate": "1994-07-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " was furiously. deposits among the ironic" }
-{ "l_orderkey": 4261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 38670.12, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly pendi" }
-{ "l_orderkey": 4359, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s affix sly" }
-{ "l_orderkey": 4613, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "against the quickly r" }
-{ "l_orderkey": 5606, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50485.99, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "carefully final foxes. pending, final" }
-{ "l_orderkey": 66, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-01", "l_receiptdate": "1994-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular de" }
-{ "l_orderkey": 1287, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-09-12", "l_receiptdate": "1994-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s wake unusual grou" }
-{ "l_orderkey": 1991, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6445.02, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hes nag slyly" }
-{ "l_orderkey": 2054, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15038.38, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uickly final" }
-{ "l_orderkey": 3687, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10741.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-03-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ing pinto beans" }
-{ "l_orderkey": 3719, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2148.34, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-04-25", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ccounts boost carefu" }
-{ "l_orderkey": 5923, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 49411.82, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "nto beans cajole blithe" }
-{ "l_orderkey": 67, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5370.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-20", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y unusual packages thrash pinto " }
-{ "l_orderkey": 261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30076.76, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-24", "l_commitdate": "1993-08-20", "l_receiptdate": "1993-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ironic packages nag slyly. carefully fin" }
-{ "l_orderkey": 775, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22557.57, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " quickly sile" }
-{ "l_orderkey": 1606, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37595.95, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully sil" }
-{ "l_orderkey": 1767, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25780.08, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-16", "l_commitdate": "1995-04-29", "l_receiptdate": "1995-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "luffy theodolites need to detect furi" }
-{ "l_orderkey": 2213, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 41892.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "the blithely " }
-{ "l_orderkey": 3360, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. carefully even deposits wake acros" }
-{ "l_orderkey": 4097, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45115.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "carefully silent foxes are against the " }
-{ "l_orderkey": 5063, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46189.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "latelets might nod blithely regular requ" }
-{ "l_orderkey": 5765, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51560.16, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "theodolites integrate furiously" }
-{ "l_orderkey": 289, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26854.25, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "out the quickly bold theodol" }
-{ "l_orderkey": 580, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ose alongside of the sl" }
-{ "l_orderkey": 871, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4296.68, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-01-20", "l_receiptdate": "1996-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l, regular dependencies w" }
-{ "l_orderkey": 1857, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular, regular inst" }
 { "l_orderkey": 2950, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 29002.59, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-10-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "are alongside of the carefully silent " }
 { "l_orderkey": 3010, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23631.74, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-06", "l_commitdate": "1996-04-06", "l_receiptdate": "1996-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final deposit" }
+{ "l_orderkey": 3360, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 33299.27, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. carefully even deposits wake acros" }
+{ "l_orderkey": 3687, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10741.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-03-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ing pinto beans" }
+{ "l_orderkey": 3719, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2148.34, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1997-04-25", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ccounts boost carefu" }
+{ "l_orderkey": 4097, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 45115.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "carefully silent foxes are against the " }
+{ "l_orderkey": 4261, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 38670.12, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly pendi" }
+{ "l_orderkey": 4359, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44040.97, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s affix sly" }
 { "l_orderkey": 4450, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 47263.48, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-12", "l_commitdate": "1997-10-13", "l_receiptdate": "1997-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " the slyly eve" }
-{ "l_orderkey": 2466, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20390.23, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts cajole a" }
-{ "l_orderkey": 3238, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27902.42, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g accounts sleep furiously ironic attai" }
-{ "l_orderkey": 4545, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40780.46, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nts serve according to th" }
-{ "l_orderkey": 4608, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-10-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s cajole. slyly " }
-{ "l_orderkey": 5377, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 28975.59, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "press theodolites. e" }
+{ "l_orderkey": 4613, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16112.55, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "against the quickly r" }
+{ "l_orderkey": 5063, "l_partkey": 174, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 46189.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "latelets might nod blithely regular requ" }
+{ "l_orderkey": 5606, "l_partkey": 174, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50485.99, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "carefully final foxes. pending, final" }
+{ "l_orderkey": 5765, "l_partkey": 174, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51560.16, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "theodolites integrate furiously" }
+{ "l_orderkey": 5923, "l_partkey": 174, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 49411.82, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "nto beans cajole blithe" }
 { "l_orderkey": 448, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49365.82, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-31", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " to the fluffily ironic packages." }
 { "l_orderkey": 672, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43999.97, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-20", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " dependencies in" }
+{ "l_orderkey": 995, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16097.55, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-30", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses. fluffily fina" }
+{ "l_orderkey": 996, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 46146.31, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " the blithely ironic foxes. slyly silent d" }
+{ "l_orderkey": 1312, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19317.06, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-07-08", "l_receiptdate": "1994-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly ironic" }
 { "l_orderkey": 1408, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7512.19, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-14", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fully final instructions. theodolites ca" }
 { "l_orderkey": 1540, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40780.46, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " final grouches bo" }
+{ "l_orderkey": 1664, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ess multip" }
+{ "l_orderkey": 2466, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 20390.23, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts cajole a" }
+{ "l_orderkey": 2534, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 18243.89, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-08-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "riously regular " }
+{ "l_orderkey": 2561, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "larly pending t" }
 { "l_orderkey": 2630, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 48292.65, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-01-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "edly express ideas. carefully final " }
+{ "l_orderkey": 2853, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42926.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly. pearls cajole. final accounts ca" }
 { "l_orderkey": 2945, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10731.7, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-03-10", "l_receiptdate": "1996-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thely. final courts could hang qu" }
+{ "l_orderkey": 3238, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27902.42, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g accounts sleep furiously ironic attai" }
+{ "l_orderkey": 3398, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1073.17, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " blithely final deposits." }
 { "l_orderkey": 3495, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25756.08, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-10", "l_receiptdate": "1996-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ic, final pains along the even request" }
+{ "l_orderkey": 3654, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 48292.65, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-15", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sly ironic notornis nag slyly" }
+{ "l_orderkey": 3749, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11804.87, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular requests along the " }
 { "l_orderkey": 3812, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35414.61, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal excuses d" }
 { "l_orderkey": 3814, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 15024.38, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-17", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sits along the final, ironic deposit" }
 { "l_orderkey": 3840, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7512.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-17", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". furiously final gifts sleep carefully pin" }
-{ "l_orderkey": 4738, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits are slyly! carefu" }
-{ "l_orderkey": 5153, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34341.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-09-25", "l_receiptdate": "1996-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. ironi" }
-{ "l_orderkey": 5284, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "unts detect furiously even d" }
-{ "l_orderkey": 5605, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3219.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. accounts boost. t" }
-{ "l_orderkey": 995, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16097.55, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-30", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-07-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses. fluffily fina" }
-{ "l_orderkey": 996, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 46146.31, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " the blithely ironic foxes. slyly silent d" }
-{ "l_orderkey": 1664, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ess multip" }
-{ "l_orderkey": 2853, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42926.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly. pearls cajole. final accounts ca" }
+{ "l_orderkey": 4545, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40780.46, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nts serve according to th" }
 { "l_orderkey": 4583, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 28975.59, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-12-24", "l_receiptdate": "1995-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " detect silent requests. furiously speci" }
+{ "l_orderkey": 4608, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32195.1, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-10-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s cajole. slyly " }
+{ "l_orderkey": 4738, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-20", "l_receiptdate": "1992-06-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nic deposits are slyly! carefu" }
+{ "l_orderkey": 4774, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "regular dolphins above the furi" }
 { "l_orderkey": 4869, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 45073.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-12-10", "l_receiptdate": "1994-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly even instructions. " }
 { "l_orderkey": 4964, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 30048.76, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-15", "l_receiptdate": "1997-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "among the carefully regula" }
-{ "l_orderkey": 1312, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19317.06, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-07-08", "l_receiptdate": "1994-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly ironic" }
-{ "l_orderkey": 2534, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 18243.89, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-08-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "riously regular " }
-{ "l_orderkey": 2561, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "larly pending t" }
-{ "l_orderkey": 3398, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1073.17, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " blithely final deposits." }
-{ "l_orderkey": 3654, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 48292.65, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-15", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sly ironic notornis nag slyly" }
-{ "l_orderkey": 3749, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11804.87, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular requests along the " }
-{ "l_orderkey": 4774, "l_partkey": 173, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 50438.99, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "regular dolphins above the furi" }
+{ "l_orderkey": 5153, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 34341.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-09-25", "l_receiptdate": "1996-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. ironi" }
+{ "l_orderkey": 5284, "l_partkey": 173, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17170.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "unts detect furiously even d" }
+{ "l_orderkey": 5377, "l_partkey": 173, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 28975.59, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "press theodolites. e" }
+{ "l_orderkey": 5605, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3219.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. accounts boost. t" }
 { "l_orderkey": 5664, "l_partkey": 173, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9658.53, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-08-26", "l_receiptdate": "1998-08-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ironic deposits haggle furiously. re" }
-{ "l_orderkey": 2247, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12866.04, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final accounts. requests across the furiou" }
-{ "l_orderkey": 3138, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40742.46, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely fluffily un" }
+{ "l_orderkey": 167, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28948.59, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "eans affix furiously-- packages" }
+{ "l_orderkey": 225, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4288.68, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng the ironic packages. asymptotes among " }
 { "l_orderkey": 1156, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 45031.14, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-18", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-02-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. quickly bold pains are" }
+{ "l_orderkey": 1600, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21443.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "pths sleep blithely about the" }
+{ "l_orderkey": 1634, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 47175.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "requests affix slyly. quickly even pack" }
+{ "l_orderkey": 1889, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13938.21, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to the regular accounts. carefully express" }
+{ "l_orderkey": 2247, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12866.04, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final accounts. requests across the furiou" }
 { "l_orderkey": 2817, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37525.95, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "gular foxes" }
+{ "l_orderkey": 3138, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40742.46, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely fluffily un" }
 { "l_orderkey": 3751, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39670.29, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-30", "l_commitdate": "1994-05-30", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly express courts " }
+{ "l_orderkey": 4321, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10721.7, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "wake carefully alongside of " }
 { "l_orderkey": 4448, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12866.04, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-21", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits about the ironic, bu" }
+{ "l_orderkey": 4544, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20371.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "regular ideas are furiously about" }
+{ "l_orderkey": 5255, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32165.1, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " to the silent requests cajole b" }
 { "l_orderkey": 5572, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28948.59, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " accounts. carefully final accoun" }
 { "l_orderkey": 5637, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37525.95, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-08-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s sleep blithely alongside of the ironic" }
 { "l_orderkey": 5671, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13938.21, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-02", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "bold theodolites about" }
 { "l_orderkey": 5988, "l_partkey": 172, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43958.97, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-02-06", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the pending, express reque" }
-{ "l_orderkey": 167, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28948.59, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "eans affix furiously-- packages" }
-{ "l_orderkey": 225, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4288.68, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng the ironic packages. asymptotes among " }
-{ "l_orderkey": 1634, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 47175.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "requests affix slyly. quickly even pack" }
-{ "l_orderkey": 1889, "l_partkey": 172, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13938.21, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to the regular accounts. carefully express" }
-{ "l_orderkey": 1600, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21443.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "pths sleep blithely about the" }
-{ "l_orderkey": 4321, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10721.7, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "wake carefully alongside of " }
-{ "l_orderkey": 4544, "l_partkey": 172, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20371.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "regular ideas are furiously about" }
-{ "l_orderkey": 5255, "l_partkey": 172, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32165.1, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " to the silent requests cajole b" }
 { "l_orderkey": 422, "l_partkey": 171, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10711.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-08-04", "l_receiptdate": "1997-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he furiously ironic theodolite" }
+{ "l_orderkey": 455, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11782.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "g deposits against the slyly idle foxes u" }
 { "l_orderkey": 545, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19281.06, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-01-17", "l_receiptdate": "1996-02-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "al, final packages affix. even a" }
+{ "l_orderkey": 1378, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-16", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "notornis. b" }
 { "l_orderkey": 1639, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43917.97, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-19", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "structions w" }
 { "l_orderkey": 2276, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28921.59, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-30", "l_commitdate": "1996-06-10", "l_receiptdate": "1996-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "the carefully unusual accoun" }
+{ "l_orderkey": 2310, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6427.02, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e slyly about the quickly ironic theodo" }
 { "l_orderkey": 3361, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35348.61, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uriously ironic accounts. ironic, ir" }
 { "l_orderkey": 3392, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42846.8, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-18", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ress instructions affix carefully. fur" }
+{ "l_orderkey": 3430, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 16067.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "cajole around the accounts. qui" }
 { "l_orderkey": 3811, "l_partkey": 171, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53558.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-28", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ts are slyly fluffy ideas. furiou" }
 { "l_orderkey": 3846, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 32135.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits according to the fur" }
-{ "l_orderkey": 4134, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "kly above the quickly regular " }
-{ "l_orderkey": 455, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11782.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "g deposits against the slyly idle foxes u" }
-{ "l_orderkey": 1378, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-16", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "notornis. b" }
 { "l_orderkey": 4066, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 46060.31, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-16", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "r instructions. slyly special " }
+{ "l_orderkey": 4134, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12854.04, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "kly above the quickly regular " }
 { "l_orderkey": 4546, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16067.55, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-31", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ught to cajole furiously. qu" }
-{ "l_orderkey": 5317, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19281.06, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-02", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "g to the blithely p" }
-{ "l_orderkey": 2310, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6427.02, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e slyly about the quickly ironic theodo" }
-{ "l_orderkey": 3430, "l_partkey": 171, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 16067.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "cajole around the accounts. qui" }
 { "l_orderkey": 4675, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6427.02, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-06", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual ideas thrash bl" }
+{ "l_orderkey": 5317, "l_partkey": 171, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19281.06, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-02", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "g to the blithely p" }
 { "l_orderkey": 5444, "l_partkey": 171, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 22494.57, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "aves serve sly" }
+{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
+{ "l_orderkey": 102, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "eposits cajole across" }
 { "l_orderkey": 448, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8561.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-02", "l_commitdate": "1995-10-16", "l_receiptdate": "1995-11-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts wake blithely. furiously pending" }
+{ "l_orderkey": 545, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4280.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-23", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ", ironic grouches cajole over" }
 { "l_orderkey": 645, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50297.99, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hely regular instructions alon" }
-{ "l_orderkey": 2406, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19263.06, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-02-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "azzle furiously careful" }
-{ "l_orderkey": 2752, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22473.57, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " along the quickly " }
-{ "l_orderkey": 2854, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-09-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "age carefully" }
-{ "l_orderkey": 3107, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-11-11", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "atelets must ha" }
-{ "l_orderkey": 3747, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35315.61, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular p" }
-{ "l_orderkey": 3905, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-07", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ow furiously. deposits wake ironic " }
-{ "l_orderkey": 4192, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28894.59, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully even escapades. care" }
-{ "l_orderkey": 5793, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "snooze quick" }
+{ "l_orderkey": 738, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nic, final excuses promise quickly regula" }
 { "l_orderkey": 1221, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12842.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-07", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly ironic " }
+{ "l_orderkey": 1408, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ep along the fina" }
 { "l_orderkey": 1444, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44947.14, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1995-03-03", "l_receiptdate": "1994-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold packages boost regular ideas. spe" }
 { "l_orderkey": 1634, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11771.87, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "final requests " }
 { "l_orderkey": 1954, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 31034.93, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-25", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "use thinly furiously regular asy" }
 { "l_orderkey": 2309, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14982.38, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1995-10-22", "l_receiptdate": "1996-01-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "asymptotes. furiously pending acco" }
-{ "l_orderkey": 4516, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "even pinto beans wake qui" }
-{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
-{ "l_orderkey": 102, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-07-28", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "eposits cajole across" }
-{ "l_orderkey": 545, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4280.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-23", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ", ironic grouches cajole over" }
-{ "l_orderkey": 2596, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ily special re" }
-{ "l_orderkey": 2944, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2140.34, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1997-12-01", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "luffily expr" }
-{ "l_orderkey": 4513, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31034.93, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole. regular packages boost. s" }
-{ "l_orderkey": 4960, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 44947.14, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-04-11", "l_receiptdate": "1995-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s requests cajole. " }
-{ "l_orderkey": 738, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nic, final excuses promise quickly regula" }
-{ "l_orderkey": 1408, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ep along the fina" }
 { "l_orderkey": 2342, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 53508.5, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-18", "l_receiptdate": "1996-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cial asymptotes pr" }
+{ "l_orderkey": 2406, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19263.06, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-02-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "azzle furiously careful" }
+{ "l_orderkey": 2596, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ily special re" }
+{ "l_orderkey": 2752, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22473.57, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " along the quickly " }
+{ "l_orderkey": 2854, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-09-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "age carefully" }
+{ "l_orderkey": 2944, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2140.34, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1997-12-01", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "luffily expr" }
+{ "l_orderkey": 3107, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24613.91, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-11-11", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "atelets must ha" }
+{ "l_orderkey": 3747, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 35315.61, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-11-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular p" }
 { "l_orderkey": 3874, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22473.57, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests cajole fluff" }
-{ "l_orderkey": 2279, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9622.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ns cajole after the final platelets. s" }
-{ "l_orderkey": 2309, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits alongside of the final re" }
-{ "l_orderkey": 3715, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17106.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "usly regular pearls haggle final packages" }
-{ "l_orderkey": 5317, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 32074.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-10-31", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "cross the attainments. slyly " }
+{ "l_orderkey": 3905, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6421.02, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-07", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ow furiously. deposits wake ironic " }
+{ "l_orderkey": 4192, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28894.59, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully even escapades. care" }
+{ "l_orderkey": 4513, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31034.93, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole. regular packages boost. s" }
+{ "l_orderkey": 4516, "l_partkey": 170, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36385.78, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "even pinto beans wake qui" }
+{ "l_orderkey": 4960, "l_partkey": 170, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 44947.14, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-04-11", "l_receiptdate": "1995-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s requests cajole. " }
+{ "l_orderkey": 5793, "l_partkey": 170, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 43876.97, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "snooze quick" }
+{ "l_orderkey": 129, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e carefully blithely bold dolp" }
+{ "l_orderkey": 358, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1993-11-04", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng the ironic theo" }
 { "l_orderkey": 709, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10691.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-06-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts cajole boldly " }
+{ "l_orderkey": 736, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 34213.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "iously final accoun" }
 { "l_orderkey": 865, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 36351.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-07-28", "l_receiptdate": "1993-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "furiously fluffily unusual account" }
 { "l_orderkey": 928, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31005.64, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-17", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of the s" }
 { "l_orderkey": 1057, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11760.76, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-31", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly final theodolites. furi" }
 { "l_orderkey": 1153, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 53458.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-07-13", "l_receiptdate": "1996-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic asymptotes nag slyly. " }
-{ "l_orderkey": 1894, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ily furiously bold packages. flu" }
-{ "l_orderkey": 2146, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29936.48, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-17", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "r accounts sleep furio" }
-{ "l_orderkey": 3301, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nusual, final excuses after the entici" }
-{ "l_orderkey": 3872, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19244.88, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. regular, brave accounts sleep blith" }
-{ "l_orderkey": 4578, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 44904.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-11-06", "l_receiptdate": "1993-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are caref" }
+{ "l_orderkey": 1220, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26729.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular orbi" }
+{ "l_orderkey": 1376, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23521.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "inst the final, pending " }
 { "l_orderkey": 1506, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 4276.64, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-12-06", "l_receiptdate": "1993-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "posits. furiou" }
 { "l_orderkey": 1636, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ely express reque" }
 { "l_orderkey": 1666, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 43835.56, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1996-01-04", "l_receiptdate": "1995-12-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly regular excuses; regular ac" }
 { "l_orderkey": 1732, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26729.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-15", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nag slyly. even, special de" }
+{ "l_orderkey": 1894, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ily furiously bold packages. flu" }
+{ "l_orderkey": 1959, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49181.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously ex" }
 { "l_orderkey": 2022, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40628.08, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-04-20", "l_receiptdate": "1992-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " against the express accounts wake ca" }
+{ "l_orderkey": 2146, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29936.48, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-17", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "r accounts sleep furio" }
+{ "l_orderkey": 2279, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9622.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ns cajole after the final platelets. s" }
+{ "l_orderkey": 2309, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits alongside of the final re" }
 { "l_orderkey": 2369, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 50250.52, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " to the regular dep" }
 { "l_orderkey": 2560, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43835.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-11-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " after the accounts. regular foxes are be" }
+{ "l_orderkey": 3301, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nusual, final excuses after the entici" }
+{ "l_orderkey": 3648, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14968.24, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly pending excuses. carefully i" }
 { "l_orderkey": 3713, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 48112.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al pinto beans affix after the slyly " }
+{ "l_orderkey": 3715, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17106.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "usly regular pearls haggle final packages" }
 { "l_orderkey": 3778, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29936.48, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y silent orbits print carefully against " }
 { "l_orderkey": 3810, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19244.88, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-28", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. furiously careful deposi" }
+{ "l_orderkey": 3872, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19244.88, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. regular, brave accounts sleep blith" }
+{ "l_orderkey": 4578, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 44904.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-11-06", "l_receiptdate": "1993-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are caref" }
 { "l_orderkey": 5095, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "carefully unusual plat" }
-{ "l_orderkey": 5635, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 40628.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-10-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly pendin" }
-{ "l_orderkey": 129, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 1069.16, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e carefully blithely bold dolp" }
-{ "l_orderkey": 358, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42766.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1993-11-04", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng the ironic theo" }
-{ "l_orderkey": 736, "l_partkey": 169, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 34213.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "iously final accoun" }
-{ "l_orderkey": 1220, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26729.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular orbi" }
-{ "l_orderkey": 1376, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23521.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "inst the final, pending " }
-{ "l_orderkey": 1959, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 49181.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously ex" }
-{ "l_orderkey": 3648, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14968.24, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sly pending excuses. carefully i" }
+{ "l_orderkey": 5317, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 32074.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-10-31", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "cross the attainments. slyly " }
 { "l_orderkey": 5573, "l_partkey": 169, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 45973.88, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " furiously pending packages against " }
+{ "l_orderkey": 5635, "l_partkey": 169, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 40628.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-10-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly pendin" }
 { "l_orderkey": 5953, "l_partkey": 169, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24590.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he silent ideas. silent foxes po" }
+{ "l_orderkey": 98, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10681.6, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-11-08", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " carefully. quickly ironic ideas" }
+{ "l_orderkey": 131, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48067.2, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic, bold accounts. careful" }
 { "l_orderkey": 163, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45930.88, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-19", "l_commitdate": "1997-11-19", "l_receiptdate": "1997-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al, bold dependencies wake. iron" }
-{ "l_orderkey": 677, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41658.24, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ges. furiously regular packages use " }
-{ "l_orderkey": 1315, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lites. unusual foxes affi" }
-{ "l_orderkey": 2087, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49135.36, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ter the dolphins." }
-{ "l_orderkey": 2114, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53408.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pecial pinto bean" }
-{ "l_orderkey": 4930, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29908.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e ironic, unusual courts. regula" }
-{ "l_orderkey": 5381, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18158.72, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly final requests haggle qui" }
-{ "l_orderkey": 5638, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12817.92, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "n, even requests. furiously ironic not" }
+{ "l_orderkey": 194, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 22431.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "accounts detect quickly dogged " }
+{ "l_orderkey": 359, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11749.76, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-15", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "rets wake blithely. slyly final dep" }
 { "l_orderkey": 582, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-09", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lar requests. quickly " }
+{ "l_orderkey": 677, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41658.24, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ges. furiously regular packages use " }
 { "l_orderkey": 868, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8545.28, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-08-01", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "l deposits. blithely regular pint" }
 { "l_orderkey": 903, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13886.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sleep along the final" }
-{ "l_orderkey": 1795, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-05-22", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "he always express accounts ca" }
-{ "l_orderkey": 1889, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5340.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-06-09", "l_receiptdate": "1997-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ording to the blithely silent r" }
-{ "l_orderkey": 5349, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14954.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-17", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully regular " }
 { "l_orderkey": 1121, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44862.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-03-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nts are slyly special packages. f" }
+{ "l_orderkey": 1315, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lites. unusual foxes affi" }
 { "l_orderkey": 1445, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 41658.24, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-02-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ully unusual reques" }
 { "l_orderkey": 1475, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16022.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1997-12-17", "l_receiptdate": "1998-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "xpress requests haggle after the final, fi" }
+{ "l_orderkey": 1794, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ely fluffily ironi" }
+{ "l_orderkey": 1795, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26704.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-05-22", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "he always express accounts ca" }
+{ "l_orderkey": 1889, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5340.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-06-09", "l_receiptdate": "1997-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ording to the blithely silent r" }
+{ "l_orderkey": 2050, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17090.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-07-28", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "al accounts. closely even " }
+{ "l_orderkey": 2087, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 49135.36, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ter the dolphins." }
+{ "l_orderkey": 2114, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53408.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pecial pinto bean" }
 { "l_orderkey": 2375, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3204.48, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "slyly across the furiously e" }
+{ "l_orderkey": 2434, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52339.84, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " after the requests haggle bold, fina" }
 { "l_orderkey": 2913, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 37385.6, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-08-21", "l_receiptdate": "1997-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. quickly even braids against" }
 { "l_orderkey": 2978, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 4272.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-31", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ffily unusual " }
 { "l_orderkey": 3044, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3204.48, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-27", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ecoys haggle furiously pending requests." }
 { "l_orderkey": 3654, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11749.76, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-07-30", "l_receiptdate": "1992-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "quickly along the express, ironic req" }
 { "l_orderkey": 3808, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46999.04, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-06-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the blithely regular foxes. even, final " }
+{ "l_orderkey": 3814, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "beans cajole quickly sl" }
 { "l_orderkey": 4739, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8545.28, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-22", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cording to the " }
+{ "l_orderkey": 4930, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29908.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e ironic, unusual courts. regula" }
 { "l_orderkey": 4963, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40590.08, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "tegrate daringly accou" }
 { "l_orderkey": 5093, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42726.4, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-11-04", "l_receiptdate": "1993-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ing pinto beans. quickly bold dependenci" }
-{ "l_orderkey": 5700, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25635.84, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ix carefully " }
-{ "l_orderkey": 98, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10681.6, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-11-08", "l_receiptdate": "1994-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " carefully. quickly ironic ideas" }
-{ "l_orderkey": 131, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 48067.2, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic, bold accounts. careful" }
-{ "l_orderkey": 194, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 22431.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "accounts detect quickly dogged " }
-{ "l_orderkey": 359, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11749.76, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-15", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "rets wake blithely. slyly final dep" }
-{ "l_orderkey": 1794, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ely fluffily ironi" }
-{ "l_orderkey": 2050, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17090.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-07-28", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "al accounts. closely even " }
-{ "l_orderkey": 2434, "l_partkey": 168, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 52339.84, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " after the requests haggle bold, fina" }
-{ "l_orderkey": 3814, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 38453.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "beans cajole quickly sl" }
 { "l_orderkey": 5191, "l_partkey": 168, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42726.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-04-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nes haggle sometimes. requests eng" }
+{ "l_orderkey": 5349, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14954.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-17", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully regular " }
+{ "l_orderkey": 5381, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 18158.72, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckly final requests haggle qui" }
+{ "l_orderkey": 5638, "l_partkey": 168, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12817.92, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "n, even requests. furiously ironic not" }
+{ "l_orderkey": 5700, "l_partkey": 168, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25635.84, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ix carefully " }
 { "l_orderkey": 166, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13873.08, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "fully above the blithely fina" }
-{ "l_orderkey": 579, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-25", "l_receiptdate": "1998-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "refully silent ideas cajole furious" }
-{ "l_orderkey": 2151, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24544.68, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " silent dependencies about the slyl" }
-{ "l_orderkey": 2241, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20276.04, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " are furiously quickl" }
-{ "l_orderkey": 3365, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39484.92, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-09", "l_receiptdate": "1994-11-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "oze blithely. furiously ironic theodolit" }
-{ "l_orderkey": 3520, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40552.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-09-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "yly final packages according to the quickl" }
-{ "l_orderkey": 3783, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38417.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ites haggle among the carefully unusu" }
 { "l_orderkey": 224, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12805.92, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-12", "l_commitdate": "1994-08-29", "l_receiptdate": "1994-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uriously regular packages. slyly fina" }
+{ "l_orderkey": 385, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7470.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " special asymptote" }
+{ "l_orderkey": 579, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-25", "l_receiptdate": "1998-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "refully silent ideas cajole furious" }
 { "l_orderkey": 614, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45887.88, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-02-22", "l_receiptdate": "1993-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " express accounts wake. slyly ironic ins" }
-{ "l_orderkey": 2208, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19208.88, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "packages are quickly bold de" }
-{ "l_orderkey": 2917, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the furiously silent packages. pend" }
-{ "l_orderkey": 3620, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17074.56, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. even, pending in" }
-{ "l_orderkey": 5472, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 41619.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uriously carefully " }
-{ "l_orderkey": 5767, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11738.76, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "instructions. carefully final accou" }
 { "l_orderkey": 930, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 32014.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-02-28", "l_receiptdate": "1995-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g accounts sleep along the platelets." }
 { "l_orderkey": 1412, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11738.76, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "en packages. regular packages dete" }
 { "l_orderkey": 1447, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20276.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1992-12-07", "l_receiptdate": "1993-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". quickly ironic " }
-{ "l_orderkey": 1888, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 53358.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ependencies affix blithely regular warhors" }
-{ "l_orderkey": 2563, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29880.48, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "hely regular depe" }
-{ "l_orderkey": 3457, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9604.44, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quests. foxes sleep quickly" }
-{ "l_orderkey": 3461, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25611.84, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "thely. carefully re" }
-{ "l_orderkey": 385, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7470.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " special asymptote" }
 { "l_orderkey": 1601, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6402.96, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-19", "l_commitdate": "1994-09-28", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " bold sheaves. furiously per" }
 { "l_orderkey": 1857, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42686.4, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-03-08", "l_receiptdate": "1993-02-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "slyly close d" }
+{ "l_orderkey": 1888, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 53358.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ependencies affix blithely regular warhors" }
+{ "l_orderkey": 2151, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24544.68, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-11-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " silent dependencies about the slyl" }
+{ "l_orderkey": 2208, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19208.88, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "packages are quickly bold de" }
+{ "l_orderkey": 2241, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20276.04, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " are furiously quickl" }
+{ "l_orderkey": 2563, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29880.48, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "hely regular depe" }
+{ "l_orderkey": 2917, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5335.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the furiously silent packages. pend" }
+{ "l_orderkey": 3365, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39484.92, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-09", "l_receiptdate": "1994-11-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "oze blithely. furiously ironic theodolit" }
+{ "l_orderkey": 3457, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9604.44, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quests. foxes sleep quickly" }
+{ "l_orderkey": 3461, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25611.84, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "thely. carefully re" }
+{ "l_orderkey": 3520, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40552.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-09-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "yly final packages according to the quickl" }
+{ "l_orderkey": 3620, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17074.56, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. even, pending in" }
+{ "l_orderkey": 3783, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38417.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-01-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ites haggle among the carefully unusu" }
 { "l_orderkey": 4326, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28813.32, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-29", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "inal packages. final asymptotes about t" }
 { "l_orderkey": 4421, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 49089.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-05-21", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "g dependenci" }
 { "l_orderkey": 4773, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 52290.84, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-26", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final reque" }
 { "l_orderkey": 5063, "l_partkey": 167, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2134.32, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-07-27", "l_receiptdate": "1997-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly regular i" }
 { "l_orderkey": 5157, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16007.4, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-27", "l_commitdate": "1997-08-30", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "cajole. spec" }
+{ "l_orderkey": 5472, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 41619.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uriously carefully " }
 { "l_orderkey": 5696, "l_partkey": 167, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44820.72, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "te furious" }
+{ "l_orderkey": 5767, "l_partkey": 167, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11738.76, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "instructions. carefully final accou" }
 { "l_orderkey": 5927, "l_partkey": 167, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34149.12, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1997-10-27", "l_receiptdate": "1997-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "telets. carefully bold accounts was" }
-{ "l_orderkey": 1703, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38381.76, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously express " }
-{ "l_orderkey": 2407, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. special deposits are closely." }
-{ "l_orderkey": 2691, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1066.16, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular instructions b" }
-{ "l_orderkey": 3079, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49043.36, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es. final, regula" }
-{ "l_orderkey": 3200, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28786.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "as haggle furiously against the fluff" }
-{ "l_orderkey": 4993, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-09-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " final packages at the q" }
-{ "l_orderkey": 5024, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1997-01-10", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " to the expre" }
-{ "l_orderkey": 5760, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6396.96, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " shall have to cajole along the " }
 { "l_orderkey": 227, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20257.04, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1996-01-30", "l_receiptdate": "1995-12-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole furiously a" }
+{ "l_orderkey": 359, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31984.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses detect spec" }
+{ "l_orderkey": 676, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 33050.96, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ial deposits cajo" }
 { "l_orderkey": 995, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47977.2, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-07-21", "l_receiptdate": "1995-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lar packages detect blithely above t" }
+{ "l_orderkey": 1121, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use furiously. quickly silent package" }
+{ "l_orderkey": 1698, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 15992.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final ideas. even, ironic " }
+{ "l_orderkey": 1703, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38381.76, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously express " }
+{ "l_orderkey": 1730, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43712.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-08-29", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions. unusual, even Tiresi" }
 { "l_orderkey": 1829, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6396.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s haggle! slyl" }
 { "l_orderkey": 1862, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39447.92, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l deposits. carefully even dep" }
-{ "l_orderkey": 3750, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 35183.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-15", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep blithely according to the flu" }
-{ "l_orderkey": 5253, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26654.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-21", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "urts. even theodoli" }
-{ "l_orderkey": 5605, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly. quickly pending sen" }
-{ "l_orderkey": 676, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 33050.96, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ial deposits cajo" }
-{ "l_orderkey": 1121, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-07", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " use furiously. quickly silent package" }
-{ "l_orderkey": 1730, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43712.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-08-29", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions. unusual, even Tiresi" }
-{ "l_orderkey": 2438, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29852.48, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-05", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions. bli" }
-{ "l_orderkey": 2913, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-09-25", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "requests doze quickly. furious" }
-{ "l_orderkey": 4930, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ions haggle. furiously regular ideas use " }
-{ "l_orderkey": 5095, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-14", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bold theodolites wake about the expr" }
-{ "l_orderkey": 5568, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "furious ide" }
-{ "l_orderkey": 359, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31984.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-06", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uses detect spec" }
-{ "l_orderkey": 1698, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 15992.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final ideas. even, ironic " }
 { "l_orderkey": 2021, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20257.04, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-09-05", "l_receiptdate": "1995-08-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " above the slyly fl" }
+{ "l_orderkey": 2407, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. special deposits are closely." }
+{ "l_orderkey": 2438, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29852.48, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-05", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions. bli" }
 { "l_orderkey": 2469, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11727.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ies wake carefully b" }
+{ "l_orderkey": 2691, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1066.16, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular instructions b" }
+{ "l_orderkey": 2913, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-09-25", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "requests doze quickly. furious" }
+{ "l_orderkey": 3079, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 49043.36, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es. final, regula" }
 { "l_orderkey": 3108, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27720.16, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-12", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " slyly slow foxes wake furious" }
+{ "l_orderkey": 3200, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28786.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "as haggle furiously against the fluff" }
 { "l_orderkey": 3555, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11727.76, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-10-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "oost caref" }
+{ "l_orderkey": 3750, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 35183.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-15", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep blithely according to the flu" }
 { "l_orderkey": 3777, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 19190.88, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-04", "l_commitdate": "1994-05-23", "l_receiptdate": "1994-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eful packages use slyly: even deposits " }
 { "l_orderkey": 3811, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2132.32, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly fluff" }
 { "l_orderkey": 4231, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4264.64, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-28", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lithely even packages. " }
 { "l_orderkey": 4258, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38381.76, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ns use alongs" }
+{ "l_orderkey": 4930, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-18", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ions haggle. furiously regular ideas use " }
+{ "l_orderkey": 4993, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44778.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-09-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " final packages at the q" }
+{ "l_orderkey": 5024, "l_partkey": 166, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 18124.72, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1997-01-10", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " to the expre" }
+{ "l_orderkey": 5095, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9595.44, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-14", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bold theodolites wake about the expr" }
+{ "l_orderkey": 5253, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26654.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-21", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "urts. even theodoli" }
+{ "l_orderkey": 5568, "l_partkey": 166, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "furious ide" }
+{ "l_orderkey": 5605, "l_partkey": 166, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 30918.64, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly. quickly pending sen" }
+{ "l_orderkey": 5760, "l_partkey": 166, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6396.96, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " shall have to cajole along the " }
 { "l_orderkey": 134, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37280.6, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-23", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ajole furiously. instructio" }
 { "l_orderkey": 357, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 34085.12, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-12-29", "l_receiptdate": "1997-02-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y above the carefully final accounts" }
-{ "l_orderkey": 902, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-10-12", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely even accounts poach furiously i" }
-{ "l_orderkey": 2151, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 52192.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. f" }
-{ "l_orderkey": 2307, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-23", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "olites haggle furiously around the " }
-{ "l_orderkey": 2438, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "en theodolites w" }
-{ "l_orderkey": 2657, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15977.4, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-12-18", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ole carefully above the ironic ideas. b" }
-{ "l_orderkey": 3716, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully unusual accounts. flu" }
-{ "l_orderkey": 4581, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-17", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e the blithely bold pearls ha" }
-{ "l_orderkey": 4676, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50062.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1995-10-04", "l_receiptdate": "1996-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely about the carefully special requ" }
-{ "l_orderkey": 4966, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7456.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-09", "l_receiptdate": "1997-01-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly ironic tithe" }
 { "l_orderkey": 484, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23433.52, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es are pending instructions. furiously unu" }
+{ "l_orderkey": 518, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31954.8, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-18", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly by the packages. carefull" }
+{ "l_orderkey": 902, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-10-12", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely even accounts poach furiously i" }
 { "l_orderkey": 1031, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29824.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly ironic accounts across the q" }
 { "l_orderkey": 1286, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-08-09", "l_receiptdate": "1993-06-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely bo" }
-{ "l_orderkey": 2117, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38345.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ronic accounts wake" }
-{ "l_orderkey": 2371, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 23433.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-26", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y daring accounts. regular ins" }
-{ "l_orderkey": 2848, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8521.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-21", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". silent, final ideas sublate packages. ir" }
-{ "l_orderkey": 2979, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29824.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ing, regular pinto beans. blithel" }
-{ "l_orderkey": 3168, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11716.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-03-17", "l_receiptdate": "1992-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously furious dependenc" }
-{ "l_orderkey": 3429, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-03-08", "l_receiptdate": "1997-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ites poach a" }
-{ "l_orderkey": 3746, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1994-10-25", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e of the careful" }
-{ "l_orderkey": 3748, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. blithely" }
-{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33019.96, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "gular excuses. fluffily even pinto beans c" }
-{ "l_orderkey": 518, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31954.8, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-18", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly by the packages. carefull" }
-{ "l_orderkey": 3015, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-21", "l_commitdate": "1992-11-07", "l_receiptdate": "1993-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "encies haggle furious" }
-{ "l_orderkey": 3234, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely regular f" }
-{ "l_orderkey": 3623, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g to the slyly regular packa" }
-{ "l_orderkey": 3654, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48997.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usly regular foxes. furio" }
-{ "l_orderkey": 4901, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12781.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y unusual deposits prom" }
-{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17042.56, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites " }
 { "l_orderkey": 1413, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 52192.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nstructions br" }
 { "l_orderkey": 1728, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46867.04, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ide of the slyly blithe" }
+{ "l_orderkey": 2117, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38345.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ronic accounts wake" }
+{ "l_orderkey": 2151, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 52192.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. f" }
+{ "l_orderkey": 2307, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-23", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "olites haggle furiously around the " }
+{ "l_orderkey": 2371, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 23433.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-26", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y daring accounts. regular ins" }
+{ "l_orderkey": 2438, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "en theodolites w" }
 { "l_orderkey": 2534, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-12", "l_commitdate": "1996-09-26", "l_receiptdate": "1996-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "eposits doze quickly final" }
+{ "l_orderkey": 2657, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15977.4, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-12-18", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ole carefully above the ironic ideas. b" }
+{ "l_orderkey": 2848, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8521.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-21", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". silent, final ideas sublate packages. ir" }
+{ "l_orderkey": 2979, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29824.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ing, regular pinto beans. blithel" }
+{ "l_orderkey": 3015, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-21", "l_commitdate": "1992-11-07", "l_receiptdate": "1993-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "encies haggle furious" }
+{ "l_orderkey": 3168, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11716.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-03-17", "l_receiptdate": "1992-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously furious dependenc" }
+{ "l_orderkey": 3234, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14912.24, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely regular f" }
+{ "l_orderkey": 3429, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 47932.2, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-03-08", "l_receiptdate": "1997-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ites poach a" }
+{ "l_orderkey": 3623, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 44736.72, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g to the slyly regular packa" }
+{ "l_orderkey": 3654, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48997.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usly regular foxes. furio" }
+{ "l_orderkey": 3716, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20238.04, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully unusual accounts. flu" }
+{ "l_orderkey": 3746, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1994-10-25", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e of the careful" }
+{ "l_orderkey": 3748, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25563.84, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-02", "l_receiptdate": "1998-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. blithely" }
 { "l_orderkey": 3846, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 35150.28, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-03-14", "l_receiptdate": "1998-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s instructions are. fu" }
+{ "l_orderkey": 4581, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39410.92, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-17", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e the blithely bold pearls ha" }
+{ "l_orderkey": 4676, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 50062.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1995-10-04", "l_receiptdate": "1996-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely about the carefully special requ" }
+{ "l_orderkey": 4901, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12781.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y unusual deposits prom" }
 { "l_orderkey": 4903, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6390.96, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "azzle quickly along the blithely final pla" }
+{ "l_orderkey": 4966, "l_partkey": 165, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7456.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-09", "l_receiptdate": "1997-01-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly ironic tithe" }
 { "l_orderkey": 5061, "l_partkey": 165, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19172.88, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "atelets among the ca" }
+{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17042.56, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely bold theodolites " }
+{ "l_orderkey": 5511, "l_partkey": 165, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 33019.96, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "gular excuses. fluffily even pinto beans c" }
+{ "l_orderkey": 7, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29796.48, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". slyly special requests haggl" }
+{ "l_orderkey": 101, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38309.76, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tes. blithely pending dolphins x-ray f" }
+{ "l_orderkey": 323, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-04-25", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cial requests " }
+{ "l_orderkey": 801, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12769.92, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. ironic pinto b" }
+{ "l_orderkey": 930, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " excuses among the furiously express ideas " }
+{ "l_orderkey": 1060, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11705.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e regular deposits: re" }
+{ "l_orderkey": 1184, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7449.12, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly warthogs. blithely bold foxes hag" }
+{ "l_orderkey": 1248, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ter the pending pl" }
+{ "l_orderkey": 2240, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6384.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-17", "l_receiptdate": "1992-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ymptotes boost. furiously bold p" }
 { "l_orderkey": 2372, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12769.92, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1997-12-21", "l_receiptdate": "1998-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lyly according to" }
 { "l_orderkey": 2436, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6384.96, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-25", "l_commitdate": "1995-11-30", "l_receiptdate": "1995-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "odolites. ep" }
+{ "l_orderkey": 2439, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2128.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-06-11", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "courts boos" }
+{ "l_orderkey": 2821, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-27", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "requests. blit" }
 { "l_orderkey": 2852, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30860.64, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-21", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-05-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironi" }
+{ "l_orderkey": 3591, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4256.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he final packages. deposits serve quick" }
 { "l_orderkey": 3621, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-06-18", "l_receiptdate": "1993-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " doubt about the bold deposits. carefully" }
 { "l_orderkey": 3811, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25539.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-05-16", "l_receiptdate": "1998-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "deposits. slyly regular accounts cajo" }
 { "l_orderkey": 3937, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 1064.16, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-01-08", "l_receiptdate": "1998-04-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "refully agains" }
-{ "l_orderkey": 4514, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-07-13", "l_receiptdate": "1994-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " even, silent foxes be" }
-{ "l_orderkey": 323, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-04-25", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cial requests " }
-{ "l_orderkey": 930, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-01-29", "l_receiptdate": "1995-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " excuses among the furiously express ideas " }
-{ "l_orderkey": 2439, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2128.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-06-11", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "courts boos" }
 { "l_orderkey": 4451, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42566.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-18", "l_commitdate": "1994-12-25", "l_receiptdate": "1994-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y. slyly special deposits are sly" }
-{ "l_orderkey": 5441, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3192.48, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "are. unusual, " }
-{ "l_orderkey": 5858, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48951.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-06", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "posits withi" }
-{ "l_orderkey": 7, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 29796.48, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". slyly special requests haggl" }
-{ "l_orderkey": 1184, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7449.12, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly warthogs. blithely bold foxes hag" }
-{ "l_orderkey": 2240, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6384.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-17", "l_receiptdate": "1992-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ymptotes boost. furiously bold p" }
-{ "l_orderkey": 2821, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-27", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "requests. blit" }
-{ "l_orderkey": 3591, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4256.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he final packages. deposits serve quick" }
+{ "l_orderkey": 4514, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28732.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-07-13", "l_receiptdate": "1994-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " even, silent foxes be" }
 { "l_orderkey": 5092, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31924.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-12-08", "l_receiptdate": "1996-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ss, ironic deposits. furiously stea" }
-{ "l_orderkey": 101, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38309.76, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tes. blithely pending dolphins x-ray f" }
-{ "l_orderkey": 801, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12769.92, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. ironic pinto b" }
-{ "l_orderkey": 1060, "l_partkey": 164, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11705.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e regular deposits: re" }
-{ "l_orderkey": 1248, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ter the pending pl" }
+{ "l_orderkey": 5441, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3192.48, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "are. unusual, " }
 { "l_orderkey": 5601, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 47887.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-25", "l_commitdate": "1992-04-03", "l_receiptdate": "1992-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts-- blithely final accounts cajole. carefu" }
 { "l_orderkey": 5633, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53208.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts. slyly regular " }
 { "l_orderkey": 5827, "l_partkey": 164, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3192.48, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-09-29", "l_receiptdate": "1998-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uses eat along the furiously" }
+{ "l_orderkey": 5858, "l_partkey": 164, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48951.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-06", "l_receiptdate": "1992-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "posits withi" }
 { "l_orderkey": 643, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24452.68, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-09", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits are carefully according to the e" }
+{ "l_orderkey": 676, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37210.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-13", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "into beans. blithe" }
+{ "l_orderkey": 710, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49968.52, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "usual ideas into th" }
+{ "l_orderkey": 997, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-16", "l_commitdate": "1997-07-21", "l_receiptdate": "1997-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "p furiously according to t" }
+{ "l_orderkey": 1316, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8505.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages against the express requests wa" }
+{ "l_orderkey": 1795, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly. special pa" }
+{ "l_orderkey": 2246, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13821.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-15", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests. fluffily special epitaphs use" }
 { "l_orderkey": 2502, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 35084.28, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-07-22", "l_receiptdate": "1993-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "have to print" }
 { "l_orderkey": 2789, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17010.56, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-18", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "o beans use carefully" }
-{ "l_orderkey": 4099, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51031.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle according to the slyly f" }
-{ "l_orderkey": 4258, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9568.44, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts wake permanently after the bravely" }
-{ "l_orderkey": 5698, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " asymptotes sleep slyly above the" }
-{ "l_orderkey": 676, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 37210.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-13", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "into beans. blithe" }
-{ "l_orderkey": 997, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-16", "l_commitdate": "1997-07-21", "l_receiptdate": "1997-07-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "p furiously according to t" }
-{ "l_orderkey": 1795, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly. special pa" }
-{ "l_orderkey": 3718, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17010.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-11", "l_commitdate": "1996-12-25", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly even accounts. blithely special acco" }
-{ "l_orderkey": 4064, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-31", "l_receiptdate": "1997-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly regular ideas." }
-{ "l_orderkey": 4292, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 42526.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ounts according to the furiously " }
-{ "l_orderkey": 4326, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press reque" }
-{ "l_orderkey": 4705, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29768.48, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes wake according to the unusual plate" }
-{ "l_orderkey": 710, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49968.52, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "usual ideas into th" }
-{ "l_orderkey": 2246, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13821.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-15", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests. fluffily special epitaphs use" }
+{ "l_orderkey": 2914, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-05-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cross the carefully even accounts." }
 { "l_orderkey": 2983, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 46779.04, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-09", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly regular instruct" }
+{ "l_orderkey": 3014, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38273.76, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-20", "l_receiptdate": "1992-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ding accounts boost fu" }
 { "l_orderkey": 3520, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37210.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-09-03", "l_receiptdate": "1997-09-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s nag carefully. sometimes unusual account" }
 { "l_orderkey": 3652, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 41463.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-10", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-03-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y express instructions. un" }
-{ "l_orderkey": 4672, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y fluffily stealt" }
-{ "l_orderkey": 5254, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47842.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-09-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " wake. blithely silent excuse" }
-{ "l_orderkey": 1316, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8505.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages against the express requests wa" }
-{ "l_orderkey": 2914, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-05-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cross the carefully even accounts." }
-{ "l_orderkey": 3014, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38273.76, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-20", "l_receiptdate": "1992-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ding accounts boost fu" }
 { "l_orderkey": 3684, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20200.04, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-19", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly carefully pending foxes. d" }
+{ "l_orderkey": 3718, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 17010.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-11", "l_commitdate": "1996-12-25", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "slyly even accounts. blithely special acco" }
 { "l_orderkey": 3841, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 51031.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-22", "l_receiptdate": "1994-12-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " according to the regular, " }
+{ "l_orderkey": 4064, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-31", "l_receiptdate": "1997-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly regular ideas." }
+{ "l_orderkey": 4099, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 51031.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle according to the slyly f" }
+{ "l_orderkey": 4258, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9568.44, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts wake permanently after the bravely" }
+{ "l_orderkey": 4292, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 42526.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ounts according to the furiously " }
+{ "l_orderkey": 4326, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11694.76, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press reque" }
 { "l_orderkey": 4578, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 21263.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-11", "l_commitdate": "1992-11-09", "l_receiptdate": "1993-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "iously pending theodolites--" }
+{ "l_orderkey": 4672, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25515.84, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y fluffily stealt" }
+{ "l_orderkey": 4705, "l_partkey": 163, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29768.48, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes wake according to the unusual plate" }
 { "l_orderkey": 4992, "l_partkey": 163, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46779.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-06-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "rmanent, sly packages print slyly. regula" }
+{ "l_orderkey": 5254, "l_partkey": 163, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47842.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-09-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " wake. blithely silent excuse" }
+{ "l_orderkey": 5698, "l_partkey": 163, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26579.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-08-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " asymptotes sleep slyly above the" }
+{ "l_orderkey": 35, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36113.44, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s are carefully against the f" }
 { "l_orderkey": 165, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45672.88, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "jole slyly according " }
+{ "l_orderkey": 192, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tes. carefu" }
+{ "l_orderkey": 258, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47797.2, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular excuses-- fluffily ruthl" }
+{ "l_orderkey": 259, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14870.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-11-20", "l_receiptdate": "1993-11-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully even, regul" }
+{ "l_orderkey": 288, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32926.96, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ns. fluffily" }
 { "l_orderkey": 358, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33989.12, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lyly express deposits " }
+{ "l_orderkey": 420, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly against the blithely re" }
+{ "l_orderkey": 422, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26554.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-07-09", "l_receiptdate": "1997-09-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ep along the furiousl" }
 { "l_orderkey": 450, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44610.72, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-05-29", "l_receiptdate": "1995-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y asymptotes. regular depen" }
 { "l_orderkey": 551, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-08-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y along the carefully ex" }
 { "l_orderkey": 833, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9559.44, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1994-04-26", "l_receiptdate": "1994-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ecial, even requests. even, bold instructi" }
+{ "l_orderkey": 1122, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25491.84, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely requests. slyly pending r" }
+{ "l_orderkey": 1382, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19118.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-10-19", "l_receiptdate": "1993-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "hely regular deposits. fluffy s" }
+{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15932.4, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "pinto beans cajole. bravely bold" }
+{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9559.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular dependencies wake. blithely final e" }
 { "l_orderkey": 2114, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26554.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-15", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "unts. regular, express accounts wake. b" }
 { "l_orderkey": 2273, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19118.88, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1997-01-21", "l_receiptdate": "1997-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " beans. doggedly final packages wake" }
+{ "l_orderkey": 2432, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8497.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-16", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s about the bold, close deposit" }
 { "l_orderkey": 2470, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31864.8, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s across the furiously fina" }
-{ "l_orderkey": 5089, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ironic accounts" }
-{ "l_orderkey": 5505, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48859.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1997-11-04", "l_receiptdate": "1998-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usly ironic dependencies haggle across " }
-{ "l_orderkey": 35, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 36113.44, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s are carefully against the f" }
-{ "l_orderkey": 192, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tes. carefu" }
-{ "l_orderkey": 259, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14870.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-11-20", "l_receiptdate": "1993-11-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully even, regul" }
+{ "l_orderkey": 2630, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30802.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully unusual dependencies. even i" }
+{ "l_orderkey": 2691, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-06-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the even foxes. unusual theodoli" }
 { "l_orderkey": 2723, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2124.32, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " courts boost quickly about th" }
+{ "l_orderkey": 2786, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-03", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "slow instructi" }
 { "l_orderkey": 3527, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 53108.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-03", "l_receiptdate": "1997-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e even accounts was about th" }
 { "l_orderkey": 3687, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20181.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly final asymptotes according to t" }
 { "l_orderkey": 3842, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29740.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s excuses thrash carefully." }
-{ "l_orderkey": 4865, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits haggle. fur" }
-{ "l_orderkey": 5543, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "instructions. deposits use quickly. ir" }
-{ "l_orderkey": 5638, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-03-27", "l_receiptdate": "1994-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press courts use f" }
-{ "l_orderkey": 5953, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5310.8, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-04-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s. blithely " }
-{ "l_orderkey": 288, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32926.96, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ns. fluffily" }
-{ "l_orderkey": 420, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly against the blithely re" }
-{ "l_orderkey": 1382, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19118.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-10-19", "l_receiptdate": "1993-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "hely regular deposits. fluffy s" }
-{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9559.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular dependencies wake. blithely final e" }
-{ "l_orderkey": 2691, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-06-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "bove the even foxes. unusual theodoli" }
-{ "l_orderkey": 2786, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-03", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "slow instructi" }
 { "l_orderkey": 4258, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20181.04, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly busily ironic foxes. f" }
-{ "l_orderkey": 4836, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans. care" }
-{ "l_orderkey": 4961, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43548.56, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ily against the n" }
-{ "l_orderkey": 5510, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 49921.52, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously even requests. slyly bold accou" }
-{ "l_orderkey": 258, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47797.2, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular excuses-- fluffily ruthl" }
-{ "l_orderkey": 422, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26554.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-07-09", "l_receiptdate": "1997-09-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ep along the furiousl" }
-{ "l_orderkey": 1122, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25491.84, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely requests. slyly pending r" }
-{ "l_orderkey": 1730, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15932.4, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "pinto beans cajole. bravely bold" }
-{ "l_orderkey": 2432, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8497.28, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-16", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s about the bold, close deposit" }
-{ "l_orderkey": 2630, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30802.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully unusual dependencies. even i" }
 { "l_orderkey": 4262, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5310.8, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-16", "l_receiptdate": "1996-10-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ironic accounts are unusu" }
 { "l_orderkey": 4512, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31864.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly unusual package" }
+{ "l_orderkey": 4836, "l_partkey": 162, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans. care" }
+{ "l_orderkey": 4865, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16994.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits haggle. fur" }
+{ "l_orderkey": 4961, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43548.56, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ily against the n" }
+{ "l_orderkey": 5089, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21243.2, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ironic accounts" }
 { "l_orderkey": 5346, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 37175.6, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-03-01", "l_receiptdate": "1994-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nic excuses cajole entic" }
+{ "l_orderkey": 5505, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48859.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1997-11-04", "l_receiptdate": "1998-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usly ironic dependencies haggle across " }
+{ "l_orderkey": 5510, "l_partkey": 162, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 49921.52, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "riously even requests. slyly bold accou" }
+{ "l_orderkey": 5543, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23367.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "instructions. deposits use quickly. ir" }
 { "l_orderkey": 5635, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24429.68, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-11-10", "l_receiptdate": "1992-09-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily pending packages. bold," }
+{ "l_orderkey": 5638, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 22305.36, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-03-27", "l_receiptdate": "1994-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "press courts use f" }
 { "l_orderkey": 5765, "l_partkey": 162, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32926.96, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r foxes. ev" }
+{ "l_orderkey": 5953, "l_partkey": 162, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5310.8, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-04-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s. blithely " }
+{ "l_orderkey": 771, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40324.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " quickly final requests are final packages." }
 { "l_orderkey": 898, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9550.44, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-04", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e slyly across the blithe" }
 { "l_orderkey": 967, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-06", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ngage blith" }
+{ "l_orderkey": 1092, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29712.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "affix carefully. u" }
+{ "l_orderkey": 1121, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28651.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts cajole slyly abou" }
 { "l_orderkey": 1315, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20162.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-05", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-08-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nal, regular warhorses about the fu" }
 { "l_orderkey": 1383, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20162.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-24", "l_commitdate": "1993-07-07", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lyly unusual accounts sle" }
+{ "l_orderkey": 1732, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43507.56, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "quests sublate against the silent " }
+{ "l_orderkey": 1765, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38201.76, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-03-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "he blithely pending accou" }
 { "l_orderkey": 1895, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45629.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-26", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-08-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " carefully eve" }
+{ "l_orderkey": 2240, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30773.64, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lyly even ideas w" }
+{ "l_orderkey": 2273, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21223.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "cuses. quickly enticing requests wake " }
+{ "l_orderkey": 2407, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7428.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-11", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes are carefully accordin" }
 { "l_orderkey": 2438, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 28651.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-10-01", "l_receiptdate": "1993-08-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "inal accounts. slyly final reques" }
 { "l_orderkey": 2593, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 46691.04, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-10-23", "l_receiptdate": "1993-09-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ents impress furiously; unusual theodoli" }
+{ "l_orderkey": 2817, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4244.64, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-04", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "n accounts wake across the fluf" }
 { "l_orderkey": 3012, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39262.92, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-16", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uickly permanent packages sleep caref" }
 { "l_orderkey": 3269, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42446.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "es. pending d" }
 { "l_orderkey": 3552, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 38201.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-24", "l_receiptdate": "1997-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly regular theodolites. fin" }
 { "l_orderkey": 3618, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27590.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-23", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "iously regular deposits cajole ruthless" }
+{ "l_orderkey": 4004, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46691.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ut the sauternes. bold, ironi" }
+{ "l_orderkey": 4391, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ong the silent deposits" }
+{ "l_orderkey": 4645, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37140.6, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sias believe bl" }
+{ "l_orderkey": 4871, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18039.72, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-09", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es. carefully ev" }
 { "l_orderkey": 4935, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13795.08, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-20", "l_commitdate": "1993-08-13", "l_receiptdate": "1993-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly requests. final deposits might " }
+{ "l_orderkey": 5031, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42446.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ns hang blithely across th" }
 { "l_orderkey": 5060, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15917.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-08-20", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ular deposits sl" }
 { "l_orderkey": 5062, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-25", "l_receiptdate": "1992-11-05", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "furiously pending requests are ruthles" }
-{ "l_orderkey": 1121, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28651.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts cajole slyly abou" }
-{ "l_orderkey": 2817, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4244.64, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-04", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-06-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "n accounts wake across the fluf" }
-{ "l_orderkey": 4645, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 37140.6, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sias believe bl" }
-{ "l_orderkey": 771, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 40324.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " quickly final requests are final packages." }
-{ "l_orderkey": 1732, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43507.56, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "quests sublate against the silent " }
-{ "l_orderkey": 1765, "l_partkey": 161, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38201.76, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-03-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "he blithely pending accou" }
-{ "l_orderkey": 2407, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7428.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-11", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes are carefully accordin" }
-{ "l_orderkey": 4391, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ong the silent deposits" }
-{ "l_orderkey": 5031, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42446.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ns hang blithely across th" }
+{ "l_orderkey": 5063, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-10-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously special " }
 { "l_orderkey": 5415, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11672.76, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle among t" }
 { "l_orderkey": 5570, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39262.92, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-29", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-09-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y ironic pin" }
-{ "l_orderkey": 5863, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47752.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-01-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits are ab" }
-{ "l_orderkey": 1092, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 29712.48, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "affix carefully. u" }
-{ "l_orderkey": 2240, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30773.64, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lyly even ideas w" }
-{ "l_orderkey": 2273, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21223.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "cuses. quickly enticing requests wake " }
-{ "l_orderkey": 4004, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46691.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ut the sauternes. bold, ironi" }
-{ "l_orderkey": 4871, "l_partkey": 161, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18039.72, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-09", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "es. carefully ev" }
-{ "l_orderkey": 5063, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 1061.16, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-10-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously special " }
 { "l_orderkey": 5858, "l_partkey": 161, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19100.88, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-12-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "al excuses. bold" }
-{ "l_orderkey": 1220, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38165.76, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar packages. blithely final acc" }
-{ "l_orderkey": 2535, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20143.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ructions. final requests" }
-{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1060.16, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final excuses. carefully even waters hagg" }
-{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11661.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual re" }
-{ "l_orderkey": 3584, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24383.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-10", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l platelets until the asymptotes " }
-{ "l_orderkey": 4262, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 43466.56, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cuses unwind ac" }
-{ "l_orderkey": 4389, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ironic request" }
-{ "l_orderkey": 4807, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es use final excuses. furiously final" }
-{ "l_orderkey": 5506, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6360.96, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hely according to the furiously unusua" }
+{ "l_orderkey": 5863, "l_partkey": 161, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47752.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-01-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits are ab" }
 { "l_orderkey": 645, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-01-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heodolites b" }
 { "l_orderkey": 710, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 48767.36, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ges use; blithely pending excuses inte" }
 { "l_orderkey": 769, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4240.64, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-08-12", "l_receiptdate": "1993-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ideas. even" }
+{ "l_orderkey": 806, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-08-11", "l_receiptdate": "1996-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fily pending " }
+{ "l_orderkey": 1157, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14842.24, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites. fluffily re" }
+{ "l_orderkey": 1220, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 38165.76, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar packages. blithely final acc" }
 { "l_orderkey": 1282, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20143.04, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-04-07", "l_receiptdate": "1992-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts x-ray across the furi" }
+{ "l_orderkey": 1317, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pinto beans according to the final, pend" }
+{ "l_orderkey": 1346, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the pinto " }
 { "l_orderkey": 1409, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 18022.72, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-04-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "pending accounts poach. care" }
 { "l_orderkey": 1441, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 39225.92, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-26", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-04-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts. slyly special dolphins b" }
 { "l_orderkey": 1888, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 4240.64, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lphins. ironically special theodolit" }
+{ "l_orderkey": 2118, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25443.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the slyly bold depende" }
 { "l_orderkey": 2176, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-03-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ruthless deposits according to the ent" }
 { "l_orderkey": 2211, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular, express" }
 { "l_orderkey": 2374, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25443.84, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". requests are above t" }
-{ "l_orderkey": 2562, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "eep against the furiously r" }
-{ "l_orderkey": 3939, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8481.28, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-04-05", "l_receiptdate": "1996-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e packages. express, pen" }
-{ "l_orderkey": 5863, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22263.36, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "atelets nag blithely furi" }
-{ "l_orderkey": 806, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-08-11", "l_receiptdate": "1996-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fily pending " }
-{ "l_orderkey": 1346, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the pinto " }
-{ "l_orderkey": 2118, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25443.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the slyly bold depende" }
 { "l_orderkey": 2532, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-23", "l_commitdate": "1996-01-04", "l_receiptdate": "1995-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rve carefully slyly ironic accounts! fluf" }
-{ "l_orderkey": 3364, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-09", "l_commitdate": "1997-08-01", "l_receiptdate": "1997-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously regular ideas haggle furiously b" }
-{ "l_orderkey": 4224, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53008.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-10", "l_receiptdate": "1997-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "side of the carefully silent dep" }
-{ "l_orderkey": 4867, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3180.48, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "yly silent deposits" }
-{ "l_orderkey": 5125, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " thinly even pack" }
-{ "l_orderkey": 5504, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-13", "l_receiptdate": "1993-02-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ajole carefully. care" }
-{ "l_orderkey": 1157, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14842.24, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites. fluffily re" }
-{ "l_orderkey": 1317, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pinto beans according to the final, pend" }
+{ "l_orderkey": 2535, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 20143.04, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ructions. final requests" }
+{ "l_orderkey": 2562, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "eep against the furiously r" }
 { "l_orderkey": 2823, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19082.88, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final deposits. furiously regular foxes u" }
 { "l_orderkey": 2854, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 21203.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "rs impress after the deposits. " }
+{ "l_orderkey": 3364, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7421.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-09", "l_commitdate": "1997-08-01", "l_receiptdate": "1997-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously regular ideas haggle furiously b" }
+{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1060.16, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final excuses. carefully even waters hagg" }
+{ "l_orderkey": 3488, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11661.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual re" }
+{ "l_orderkey": 3584, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24383.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-10", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l platelets until the asymptotes " }
+{ "l_orderkey": 3939, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8481.28, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-04-05", "l_receiptdate": "1996-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e packages. express, pen" }
 { "l_orderkey": 4135, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34985.28, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-05-23", "l_receiptdate": "1997-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he fluffil" }
+{ "l_orderkey": 4224, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 53008.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-10", "l_receiptdate": "1997-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "side of the carefully silent dep" }
+{ "l_orderkey": 4262, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 43466.56, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cuses unwind ac" }
 { "l_orderkey": 4325, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19082.88, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-09-28", "l_receiptdate": "1996-10-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ". blithely" }
+{ "l_orderkey": 4389, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ironic request" }
 { "l_orderkey": 4454, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21203.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "quickly regular requests. furiously" }
+{ "l_orderkey": 4807, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 23323.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es use final excuses. furiously final" }
+{ "l_orderkey": 4867, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3180.48, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "yly silent deposits" }
+{ "l_orderkey": 5125, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5300.8, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " thinly even pack" }
 { "l_orderkey": 5443, "l_partkey": 160, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26504.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1997-01-08", "l_receiptdate": "1997-01-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "use carefully above the pinto bea" }
+{ "l_orderkey": 5504, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-13", "l_receiptdate": "1993-02-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ajole carefully. care" }
+{ "l_orderkey": 5506, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6360.96, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hely according to the furiously unusua" }
 { "l_orderkey": 5633, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29684.48, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "as boost quickly. unusual pinto " }
 { "l_orderkey": 5830, "l_partkey": 160, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30744.64, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y bold excuses" }
+{ "l_orderkey": 5863, "l_partkey": 160, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 22263.36, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "atelets nag blithely furi" }
 { "l_orderkey": 5925, "l_partkey": 160, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 43466.56, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " across the pending deposits nag caref" }
+{ "l_orderkey": 231, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e furiously ironic pinto beans." }
+{ "l_orderkey": 325, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36011.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly bold deposits. always iron" }
 { "l_orderkey": 519, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1059.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold requests believe furiou" }
+{ "l_orderkey": 551, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21183.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "r ideas. final, even ideas hinder alongside" }
+{ "l_orderkey": 613, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7414.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-09-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ously blithely final pinto beans. regula" }
+{ "l_orderkey": 1222, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-05", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously bold instructions" }
 { "l_orderkey": 1315, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33892.8, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "neath the final p" }
+{ "l_orderkey": 1954, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ongside of the slyly unusual requests. reg" }
 { "l_orderkey": 1955, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11650.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-06-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ously quickly pendi" }
+{ "l_orderkey": 2277, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32833.65, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-07", "l_commitdate": "1995-03-19", "l_receiptdate": "1995-03-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ic instructions detect ru" }
+{ "l_orderkey": 2371, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39188.55, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-11", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s boost fluffil" }
+{ "l_orderkey": 2468, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19064.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-26", "l_receiptdate": "1997-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cies. fluffily r" }
 { "l_orderkey": 2499, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 41306.85, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-27", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "otes sublat" }
+{ "l_orderkey": 2595, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30715.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic accounts haggle carefully fin" }
+{ "l_orderkey": 3363, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1995-12-01", "l_receiptdate": "1996-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uickly bold ide" }
 { "l_orderkey": 3588, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47661.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-07", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ecial pains integrate blithely. reques" }
+{ "l_orderkey": 3714, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ccounts cajole fu" }
 { "l_orderkey": 3776, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14828.1, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-30", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y special ideas. express packages pr" }
 { "l_orderkey": 3938, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48720.9, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-05-04", "l_receiptdate": "1993-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly even foxes are slyly fu" }
 { "l_orderkey": 4006, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 19064.7, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-03-08", "l_receiptdate": "1995-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "gouts! slyly iron" }
-{ "l_orderkey": 5062, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52957.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the regular, unusual pains. specia" }
-{ "l_orderkey": 5825, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24360.45, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-04-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " special pinto beans. dependencies haggl" }
-{ "l_orderkey": 5957, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33892.8, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " boost carefully across the " }
-{ "l_orderkey": 231, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e furiously ironic pinto beans." }
-{ "l_orderkey": 551, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21183.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "r ideas. final, even ideas hinder alongside" }
-{ "l_orderkey": 1954, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ongside of the slyly unusual requests. reg" }
-{ "l_orderkey": 2371, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 39188.55, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-11", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s boost fluffil" }
 { "l_orderkey": 4451, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20123.85, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-11-26", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly after the fluffi" }
+{ "l_orderkey": 4549, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ding to the regular, silent requests" }
+{ "l_orderkey": 4610, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " final theodolites " }
 { "l_orderkey": 4738, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10591.5, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-10", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "hins above the" }
+{ "l_orderkey": 4743, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3177.45, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "al requests. express idea" }
+{ "l_orderkey": 4807, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas wake bli" }
+{ "l_orderkey": 5062, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52957.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the regular, unusual pains. specia" }
 { "l_orderkey": 5092, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 52957.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-30", "l_commitdate": "1996-01-14", "l_receiptdate": "1995-12-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r platelets maintain car" }
 { "l_orderkey": 5409, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39188.55, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-02-10", "l_receiptdate": "1992-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ously regular packages. packages" }
 { "l_orderkey": 5728, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42366.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-28", "l_commitdate": "1995-01-17", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "final deposits. theodolite" }
-{ "l_orderkey": 325, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 36011.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly bold deposits. always iron" }
-{ "l_orderkey": 2277, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32833.65, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-07", "l_commitdate": "1995-03-19", "l_receiptdate": "1995-03-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ic instructions detect ru" }
-{ "l_orderkey": 2595, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30715.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic accounts haggle carefully fin" }
-{ "l_orderkey": 3363, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1995-12-01", "l_receiptdate": "1996-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uickly bold ide" }
-{ "l_orderkey": 4549, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ding to the regular, silent requests" }
-{ "l_orderkey": 4610, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46602.6, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " final theodolites " }
-{ "l_orderkey": 613, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7414.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-09-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ously blithely final pinto beans. regula" }
-{ "l_orderkey": 1222, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12709.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-05", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously bold instructions" }
-{ "l_orderkey": 2468, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19064.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-26", "l_receiptdate": "1997-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cies. fluffily r" }
-{ "l_orderkey": 3714, "l_partkey": 159, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16946.4, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ccounts cajole fu" }
-{ "l_orderkey": 4743, "l_partkey": 159, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3177.45, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "al requests. express idea" }
-{ "l_orderkey": 4807, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2118.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas wake bli" }
+{ "l_orderkey": 5825, "l_partkey": 159, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24360.45, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-04-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " special pinto beans. dependencies haggl" }
+{ "l_orderkey": 5957, "l_partkey": 159, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33892.8, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " boost carefully across the " }
+{ "l_orderkey": 7, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ithely regula" }
 { "l_orderkey": 135, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34918.95, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ptotes boost slowly care" }
+{ "l_orderkey": 449, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23279.3, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "furiously final theodolites eat careful" }
+{ "l_orderkey": 736, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48674.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-01", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uctions cajole" }
+{ "l_orderkey": 839, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24337.45, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng ideas haggle accord" }
+{ "l_orderkey": 1317, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 27511.9, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "leep along th" }
+{ "l_orderkey": 1412, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11639.65, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-04-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se slyly. special, unusual accounts nag bl" }
+{ "l_orderkey": 1859, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "across the p" }
 { "l_orderkey": 1955, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43384.15, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " carefully against the furiously reg" }
+{ "l_orderkey": 2144, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10581.5, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-16", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " furiously unusual ideas. carefull" }
 { "l_orderkey": 2567, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 52907.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "pinto beans? r" }
 { "l_orderkey": 3136, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45500.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-09-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". special theodolites ha" }
 { "l_orderkey": 3522, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 19046.7, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-16", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sits wake carefully pen" }
-{ "l_orderkey": 4773, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6348.9, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "latelets haggle s" }
-{ "l_orderkey": 839, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24337.45, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng ideas haggle accord" }
-{ "l_orderkey": 1859, "l_partkey": 158, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "across the p" }
-{ "l_orderkey": 2144, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10581.5, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-16", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " furiously unusual ideas. carefull" }
-{ "l_orderkey": 5249, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12697.8, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-07", "l_receiptdate": "1995-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "press depths could have to sleep carefu" }
-{ "l_orderkey": 5665, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43384.15, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-22", "l_receiptdate": "1993-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " idle ideas across " }
-{ "l_orderkey": 5794, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44442.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "he careful" }
-{ "l_orderkey": 449, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23279.3, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "furiously final theodolites eat careful" }
-{ "l_orderkey": 736, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 48674.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-01", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uctions cajole" }
-{ "l_orderkey": 1317, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 27511.9, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "leep along th" }
-{ "l_orderkey": 1412, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11639.65, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-04-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se slyly. special, unusual accounts nag bl" }
 { "l_orderkey": 4227, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20104.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ns sleep along the blithely even theodolit" }
+{ "l_orderkey": 4773, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6348.9, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "latelets haggle s" }
 { "l_orderkey": 4993, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32802.65, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-10-15", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nwind thinly platelets. a" }
-{ "l_orderkey": 5828, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39151.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-05-30", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully spec" }
-{ "l_orderkey": 7, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 5290.75, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ithely regula" }
 { "l_orderkey": 5089, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4232.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-09-28", "l_receiptdate": "1992-10-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nts sleep blithely " }
+{ "l_orderkey": 5249, "l_partkey": 158, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12697.8, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-07", "l_receiptdate": "1995-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "press depths could have to sleep carefu" }
 { "l_orderkey": 5254, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 35977.1, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously above the furiously " }
 { "l_orderkey": 5442, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22221.15, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ffily furiously ironic theodolites. furio" }
+{ "l_orderkey": 5665, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43384.15, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-22", "l_receiptdate": "1993-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " idle ideas across " }
 { "l_orderkey": 5669, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42326.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-06-15", "l_receiptdate": "1996-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ar accounts alongside of the final, p" }
+{ "l_orderkey": 5794, "l_partkey": 158, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44442.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "he careful" }
+{ "l_orderkey": 5828, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39151.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-05-30", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully spec" }
 { "l_orderkey": 5952, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 24337.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-05-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e blithely packages. eve" }
+{ "l_orderkey": 326, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 43343.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-05", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to beans wake before the furiously re" }
+{ "l_orderkey": 455, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44400.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "around the quickly blit" }
 { "l_orderkey": 675, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1057.15, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ide of the slyly regular packages. unus" }
+{ "l_orderkey": 802, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 19028.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y regular requests engage furiously final d" }
 { "l_orderkey": 1121, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10571.5, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dencies. quickly regular theodolites n" }
+{ "l_orderkey": 1158, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24314.45, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ularly ironic requests use care" }
 { "l_orderkey": 1382, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32771.65, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-10-15", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "hely regular dependencies. f" }
+{ "l_orderkey": 1509, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28543.05, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-08", "l_receiptdate": "1993-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely after the " }
 { "l_orderkey": 1729, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12685.8, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-07-24", "l_receiptdate": "1992-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y pending packages detect. carefully re" }
+{ "l_orderkey": 1761, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39114.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express requests print blithely around the" }
+{ "l_orderkey": 1763, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45457.45, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits integrate blithely pending, quic" }
 { "l_orderkey": 1863, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50743.2, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-08", "l_commitdate": "1993-11-05", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "onic theodolites alongside of the pending a" }
+{ "l_orderkey": 2146, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6342.9, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing to the requests. dependencies boost " }
+{ "l_orderkey": 2178, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15857.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l accounts. quickly expr" }
+{ "l_orderkey": 2433, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 40171.7, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". slyly regular requests sle" }
 { "l_orderkey": 2531, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3171.45, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-08-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he quickly ev" }
+{ "l_orderkey": 2694, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37000.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-24", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "atelets past the furiously final deposits " }
 { "l_orderkey": 3458, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2114.3, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-03-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ironic packages haggle past the furiously " }
 { "l_orderkey": 3522, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 48628.9, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-11-30", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "d the express, silent foxes. blit" }
-{ "l_orderkey": 4389, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21143.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ng the carefully express d" }
-{ "l_orderkey": 4770, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31714.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-25", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ffily carefully ironic ideas. ironic d" }
-{ "l_orderkey": 455, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44400.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-10", "l_receiptdate": "1997-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "around the quickly blit" }
-{ "l_orderkey": 1509, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 28543.05, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-08", "l_receiptdate": "1993-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely after the " }
-{ "l_orderkey": 1761, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 39114.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express requests print blithely around the" }
-{ "l_orderkey": 2178, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15857.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l accounts. quickly expr" }
+{ "l_orderkey": 3553, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 38057.4, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " realms. pending, bold theodolites " }
 { "l_orderkey": 3841, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1057.15, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-10", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " boost even re" }
 { "l_orderkey": 4069, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52857.5, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even foxes among the express wate" }
-{ "l_orderkey": 5792, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49686.05, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "regular, ironic excuses n" }
-{ "l_orderkey": 1158, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24314.45, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ularly ironic requests use care" }
-{ "l_orderkey": 1763, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45457.45, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits integrate blithely pending, quic" }
-{ "l_orderkey": 4869, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26428.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e according t" }
-{ "l_orderkey": 326, "l_partkey": 157, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 43343.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-05", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to beans wake before the furiously re" }
-{ "l_orderkey": 802, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 19028.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y regular requests engage furiously final d" }
-{ "l_orderkey": 2146, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6342.9, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-10-24", "l_receiptdate": "1993-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing to the requests. dependencies boost " }
-{ "l_orderkey": 2433, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 40171.7, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". slyly regular requests sle" }
-{ "l_orderkey": 2694, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 37000.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-24", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-05-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "atelets past the furiously final deposits " }
-{ "l_orderkey": 3553, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 38057.4, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " realms. pending, bold theodolites " }
+{ "l_orderkey": 4389, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21143.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ng the carefully express d" }
 { "l_orderkey": 4741, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 35943.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sly special packages after the furiously" }
+{ "l_orderkey": 4770, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31714.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-25", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ffily carefully ironic ideas. ironic d" }
+{ "l_orderkey": 4869, "l_partkey": 157, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26428.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e according t" }
+{ "l_orderkey": 5792, "l_partkey": 157, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49686.05, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "regular, ironic excuses n" }
 { "l_orderkey": 5922, "l_partkey": 157, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 39114.55, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1996-12-16", "l_receiptdate": "1997-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s wake slyly. requests cajole furiously asy" }
 { "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }
-{ "l_orderkey": 1378, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9505.35, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully. carefully iron" }
-{ "l_orderkey": 1735, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-14", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously after the " }
-{ "l_orderkey": 2086, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34852.95, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " slyly regular foxes. un" }
-{ "l_orderkey": 3942, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26403.75, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "d the quick packages" }
-{ "l_orderkey": 3968, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-14", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly regular accounts" }
-{ "l_orderkey": 4640, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y regular instructions doze furiously. reg" }
 { "l_orderkey": 165, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 28516.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-03-04", "l_receiptdate": "1993-05-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "around the ironic, even orb" }
+{ "l_orderkey": 197, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. careful" }
 { "l_orderkey": 229, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34852.95, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-04-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " deposits; bold, ruthless theodolites" }
 { "l_orderkey": 260, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52807.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c deposits " }
+{ "l_orderkey": 517, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " slyly. express requests ar" }
 { "l_orderkey": 578, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 42246.0, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-10", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "usly even platel" }
 { "l_orderkey": 1027, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "oxes. carefully regular deposits" }
-{ "l_orderkey": 2791, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-20", "l_receiptdate": "1995-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ilent forges. quickly special pinto beans " }
-{ "l_orderkey": 4196, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31684.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular foxes us" }
-{ "l_orderkey": 4742, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33796.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-06-12", "l_receiptdate": "1995-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits boost blithely. carefully regular a" }
-{ "l_orderkey": 4994, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38021.4, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ess ideas. blithely silent brai" }
-{ "l_orderkey": 4996, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41189.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "equests are carefully final" }
-{ "l_orderkey": 197, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. careful" }
-{ "l_orderkey": 2368, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-09-27", "l_receiptdate": "1993-10-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fily. slyly final ideas alongside o" }
-{ "l_orderkey": 2786, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43302.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ix requests. bold requests a" }
-{ "l_orderkey": 3015, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7393.05, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " after the evenly special packages ca" }
-{ "l_orderkey": 3364, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10561.5, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-10", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the accounts. final, busy accounts wi" }
-{ "l_orderkey": 4741, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "even requests." }
-{ "l_orderkey": 4995, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23235.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-12", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s wake furious, express dependencies." }
-{ "l_orderkey": 5093, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 39077.55, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "courts. qui" }
-{ "l_orderkey": 5349, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20066.85, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "endencies use whithout the special " }
-{ "l_orderkey": 5669, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2112.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-04", "l_commitdate": "1996-06-15", "l_receiptdate": "1996-08-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely excuses. slyly" }
-{ "l_orderkey": 517, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " slyly. express requests ar" }
 { "l_orderkey": 1095, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13729.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ously even accounts. slyly bold a" }
 { "l_orderkey": 1248, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51751.35, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "beans run quickly according to the carefu" }
+{ "l_orderkey": 1378, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9505.35, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully. carefully iron" }
 { "l_orderkey": 1412, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21123.0, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-04", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "odolites sleep ironically" }
 { "l_orderkey": 1700, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51751.35, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-26", "l_commitdate": "1996-07-28", "l_receiptdate": "1996-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kly even dependencies haggle fluffi" }
+{ "l_orderkey": 1735, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-14", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-02-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously after the " }
 { "l_orderkey": 1825, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " accounts breach fluffily spe" }
+{ "l_orderkey": 2086, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34852.95, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " slyly regular foxes. un" }
 { "l_orderkey": 2086, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7393.05, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1994-12-10", "l_receiptdate": "1995-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " beans haggle car" }
+{ "l_orderkey": 2368, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17954.55, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-09-27", "l_receiptdate": "1993-10-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fily. slyly final ideas alongside o" }
 { "l_orderkey": 2435, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23235.3, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-23", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e final, final deposits. carefully regular" }
+{ "l_orderkey": 2786, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 43302.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ix requests. bold requests a" }
+{ "l_orderkey": 2791, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-20", "l_receiptdate": "1995-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ilent forges. quickly special pinto beans " }
+{ "l_orderkey": 3015, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7393.05, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " after the evenly special packages ca" }
+{ "l_orderkey": 3364, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10561.5, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-10", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the accounts. final, busy accounts wi" }
 { "l_orderkey": 3488, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 19010.7, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-03-19", "l_receiptdate": "1995-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s the carefully r" }
 { "l_orderkey": 3492, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3168.45, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "the deposits. carefully " }
+{ "l_orderkey": 3942, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26403.75, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "d the quick packages" }
+{ "l_orderkey": 3968, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 45414.45, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-14", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly regular accounts" }
+{ "l_orderkey": 4196, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31684.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular foxes us" }
+{ "l_orderkey": 4640, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15842.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y regular instructions doze furiously. reg" }
+{ "l_orderkey": 4741, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 25347.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "even requests." }
+{ "l_orderkey": 4742, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33796.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-06-12", "l_receiptdate": "1995-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits boost blithely. carefully regular a" }
+{ "l_orderkey": 4994, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 38021.4, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ess ideas. blithely silent brai" }
+{ "l_orderkey": 4995, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23235.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-12", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s wake furious, express dependencies." }
+{ "l_orderkey": 4996, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 41189.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "equests are carefully final" }
+{ "l_orderkey": 5093, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 39077.55, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "courts. qui" }
 { "l_orderkey": 5348, "l_partkey": 156, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32740.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1997-12-09", "l_receiptdate": "1998-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "are finally" }
+{ "l_orderkey": 5349, "l_partkey": 156, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20066.85, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-09-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "endencies use whithout the special " }
 { "l_orderkey": 5509, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 36965.25, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-17", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "c accounts. ca" }
+{ "l_orderkey": 5669, "l_partkey": 156, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2112.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-04", "l_commitdate": "1996-06-15", "l_receiptdate": "1996-08-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely excuses. slyly" }
+{ "l_orderkey": 707, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " dependencies" }
 { "l_orderkey": 807, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51702.35, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y regular requests haggle." }
+{ "l_orderkey": 1444, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
+{ "l_orderkey": 1542, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ial instructions. ironically" }
+{ "l_orderkey": 1638, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckages are carefully even instru" }
+{ "l_orderkey": 1664, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-06", "l_commitdate": "1996-05-16", "l_receiptdate": "1996-03-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular ide" }
 { "l_orderkey": 1734, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 40095.7, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-08-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ts doubt b" }
+{ "l_orderkey": 1956, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " wake after the " }
+{ "l_orderkey": 2273, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "furiously above the ironic requests. " }
 { "l_orderkey": 2305, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 27433.9, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "arefully final theodo" }
 { "l_orderkey": 2436, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50647.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-22", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he furiously " }
+{ "l_orderkey": 2466, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages detect carefully: ironically sl" }
 { "l_orderkey": 3394, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34819.95, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-17", "l_receiptdate": "1996-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ideas alongside of th" }
-{ "l_orderkey": 4004, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9496.35, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic requests. quickly pending ide" }
-{ "l_orderkey": 4070, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42206.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "about the sentiments. quick" }
-{ "l_orderkey": 5505, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10551.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously special asym" }
-{ "l_orderkey": 1638, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckages are carefully even instru" }
-{ "l_orderkey": 2273, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "furiously above the ironic requests. " }
 { "l_orderkey": 3651, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25323.6, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-22", "l_commitdate": "1998-07-17", "l_receiptdate": "1998-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "excuses haggle according to th" }
 { "l_orderkey": 3808, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30599.35, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-22", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " deposits across the pac" }
+{ "l_orderkey": 4004, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9496.35, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic requests. quickly pending ide" }
+{ "l_orderkey": 4070, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 42206.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-08-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "about the sentiments. quick" }
 { "l_orderkey": 4742, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 30599.35, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-15", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "integrate closely among t" }
-{ "l_orderkey": 707, "l_partkey": 155, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " dependencies" }
-{ "l_orderkey": 1542, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 48536.9, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ial instructions. ironically" }
-{ "l_orderkey": 5698, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47481.75, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-23", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng excuses. slyly express asymptotes" }
-{ "l_orderkey": 1444, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35875.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
-{ "l_orderkey": 1664, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-06", "l_commitdate": "1996-05-16", "l_receiptdate": "1996-03-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular ide" }
-{ "l_orderkey": 1956, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 16882.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " wake after the " }
-{ "l_orderkey": 2466, "l_partkey": 155, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 36930.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages detect carefully: ironically sl" }
 { "l_orderkey": 5350, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7386.05, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-19", "l_commitdate": "1993-12-28", "l_receiptdate": "1993-11-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "alongside of th" }
 { "l_orderkey": 5378, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 41150.85, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-22", "l_receiptdate": "1992-12-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts are quickly around the" }
+{ "l_orderkey": 5505, "l_partkey": 155, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10551.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously special asym" }
+{ "l_orderkey": 5698, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 47481.75, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-23", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng excuses. slyly express asymptotes" }
 { "l_orderkey": 5956, "l_partkey": 155, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10551.5, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-04", "l_receiptdate": "1998-08-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ic packages am" }
-{ "l_orderkey": 608, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20028.85, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ideas. the" }
-{ "l_orderkey": 4998, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12649.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-20", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " sleep slyly furiously final accounts. ins" }
-{ "l_orderkey": 5858, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7379.05, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-14", "l_commitdate": "1992-10-01", "l_receiptdate": "1992-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dly pending ac" }
-{ "l_orderkey": 292, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8433.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-18", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sily bold deposits alongside of the ex" }
-{ "l_orderkey": 2626, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42166.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-12-03", "l_receiptdate": "1995-10-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eans. ironic deposits haggle. depo" }
-{ "l_orderkey": 3233, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6324.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-06", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "requests are quickly above the slyly p" }
-{ "l_orderkey": 3970, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10541.5, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-07-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " special packages wake after the final br" }
-{ "l_orderkey": 4032, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24245.45, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ording to the " }
-{ "l_orderkey": 4515, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28462.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " against the even re" }
-{ "l_orderkey": 5029, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-04-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! packages boost blithely. furious" }
-{ "l_orderkey": 5606, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3162.45, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-04", "l_receiptdate": "1997-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " sauternes. asympto" }
 { "l_orderkey": 193, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15812.25, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-10-09", "l_receiptdate": "1993-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ffily. regular packages d" }
-{ "l_orderkey": 1377, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-20", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s must have to mold b" }
-{ "l_orderkey": 1827, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50599.2, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-09-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oxes. special, final asymptote" }
-{ "l_orderkey": 2849, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16866.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular requ" }
-{ "l_orderkey": 3491, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29516.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-29", "l_commitdate": "1998-09-08", "l_receiptdate": "1998-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts. sly" }
-{ "l_orderkey": 3937, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 52707.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ong the carefully exp" }
-{ "l_orderkey": 4324, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 48490.9, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-03", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-11-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ular, final theodo" }
-{ "l_orderkey": 5571, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33732.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-01-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " the blithely even packages nag q" }
+{ "l_orderkey": 292, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8433.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-18", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sily bold deposits alongside of the ex" }
+{ "l_orderkey": 608, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20028.85, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ideas. the" }
 { "l_orderkey": 1281, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 40057.7, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-28", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " ideas-- blithely regular" }
 { "l_orderkey": 1377, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5270.75, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-06-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " final, final grouches. accoun" }
+{ "l_orderkey": 1377, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-20", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s must have to mold b" }
 { "l_orderkey": 1573, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 31624.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-29", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". blithely even theodolites boos" }
+{ "l_orderkey": 1827, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50599.2, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-09-15", "l_receiptdate": "1996-09-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oxes. special, final asymptote" }
 { "l_orderkey": 2145, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6324.9, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-11-29", "l_receiptdate": "1992-10-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s. fluffily express accounts sleep. slyl" }
+{ "l_orderkey": 2626, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42166.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-12-03", "l_receiptdate": "1995-10-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eans. ironic deposits haggle. depo" }
+{ "l_orderkey": 2849, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16866.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular requ" }
 { "l_orderkey": 2852, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 29516.2, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-08", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "e accounts. caref" }
 { "l_orderkey": 3168, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1054.15, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-03-12", "l_receiptdate": "1992-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pinto beans. slyly regular courts haggle " }
+{ "l_orderkey": 3233, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6324.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-06", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "requests are quickly above the slyly p" }
+{ "l_orderkey": 3491, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29516.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-29", "l_commitdate": "1998-09-08", "l_receiptdate": "1998-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts. sly" }
 { "l_orderkey": 3747, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14758.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-03", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "packages cajole carefu" }
+{ "l_orderkey": 3937, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 52707.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ong the carefully exp" }
+{ "l_orderkey": 3970, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10541.5, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-07-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " special packages wake after the final br" }
+{ "l_orderkey": 4032, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24245.45, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ording to the " }
+{ "l_orderkey": 4324, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 48490.9, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-03", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-11-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ular, final theodo" }
+{ "l_orderkey": 4515, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28462.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " against the even re" }
 { "l_orderkey": 4644, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47436.75, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-04-08", "l_receiptdate": "1998-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " carefully a" }
 { "l_orderkey": 4805, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 46382.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eposits sleep furiously qui" }
+{ "l_orderkey": 4998, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12649.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-20", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " sleep slyly furiously final accounts. ins" }
+{ "l_orderkey": 5029, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17920.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-04-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! packages boost blithely. furious" }
 { "l_orderkey": 5031, "l_partkey": 154, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4216.6, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "after the even frays: ironic, unusual th" }
 { "l_orderkey": 5538, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44274.3, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "vely ironic accounts. furiously unusual acc" }
-{ "l_orderkey": 2694, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oxes. never iro" }
-{ "l_orderkey": 2819, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5265.75, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-29", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " fluffily unusual foxes sleep caref" }
-{ "l_orderkey": 3591, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51604.35, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " mold slyly. bl" }
-{ "l_orderkey": 3717, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47391.75, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ests wake whithout the blithely final pl" }
-{ "l_orderkey": 3906, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "jole blithely after the furiously regular " }
-{ "l_orderkey": 4999, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-08-15", "l_receiptdate": "1993-08-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ades cajole carefully unusual ide" }
-{ "l_orderkey": 5184, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-10-16", "l_receiptdate": "1998-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully express asympto" }
+{ "l_orderkey": 5571, "l_partkey": 154, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33732.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-01-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " the blithely even packages nag q" }
+{ "l_orderkey": 5606, "l_partkey": 154, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3162.45, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-04", "l_receiptdate": "1997-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " sauternes. asympto" }
+{ "l_orderkey": 5858, "l_partkey": 154, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7379.05, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-14", "l_commitdate": "1992-10-01", "l_receiptdate": "1992-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dly pending ac" }
+{ "l_orderkey": 322, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12637.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular theodolites promise qu" }
+{ "l_orderkey": 386, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 41072.85, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-02-28", "l_receiptdate": "1995-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "hely. carefully regular accounts hag" }
 { "l_orderkey": 387, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lithely final theodolites." }
 { "l_orderkey": 419, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-12-25", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y above the bli" }
 { "l_orderkey": 450, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ily carefully final depo" }
 { "l_orderkey": 548, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33700.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1994-11-20", "l_receiptdate": "1994-12-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c instruction" }
+{ "l_orderkey": 647, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15797.25, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ve the even, bold foxes sleep " }
 { "l_orderkey": 1092, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1053.15, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-04-21", "l_receiptdate": "1995-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lent, pending requests-- requests nag accor" }
+{ "l_orderkey": 1347, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22116.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-10", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-11-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "g pinto beans affix car" }
 { "l_orderkey": 1636, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 45285.45, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-08-10", "l_receiptdate": "1997-09-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely special r" }
+{ "l_orderkey": 1829, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14744.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-06-08", "l_receiptdate": "1994-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular deposits alongside of the flu" }
+{ "l_orderkey": 2276, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52657.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts dete" }
+{ "l_orderkey": 2694, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oxes. never iro" }
+{ "l_orderkey": 2819, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5265.75, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-29", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " fluffily unusual foxes sleep caref" }
+{ "l_orderkey": 3591, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51604.35, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " mold slyly. bl" }
+{ "l_orderkey": 3717, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 47391.75, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ests wake whithout the blithely final pl" }
+{ "l_orderkey": 3782, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10531.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ven pinto b" }
+{ "l_orderkey": 3906, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "jole blithely after the furiously regular " }
+{ "l_orderkey": 3910, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1053.15, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s sleep neve" }
+{ "l_orderkey": 4354, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24222.45, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-23", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "kly along the ironic, ent" }
 { "l_orderkey": 4359, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8425.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages affix. fluffily regular f" }
 { "l_orderkey": 4389, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13690.95, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-06-06", "l_receiptdate": "1994-08-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nal, regula" }
 { "l_orderkey": 4422, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4212.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "cies along the bo" }
-{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38966.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ts. pinto beans use according to th" }
-{ "l_orderkey": 5382, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-22", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular accounts. even accounts integrate" }
-{ "l_orderkey": 5767, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-06-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sits among the" }
-{ "l_orderkey": 322, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12637.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular theodolites promise qu" }
-{ "l_orderkey": 386, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 41072.85, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-02-28", "l_receiptdate": "1995-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "hely. carefully regular accounts hag" }
-{ "l_orderkey": 3782, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10531.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ven pinto b" }
-{ "l_orderkey": 4354, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24222.45, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-23", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "kly along the ironic, ent" }
+{ "l_orderkey": 4455, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49498.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1993-12-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. even, even accou" }
 { "l_orderkey": 4740, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25275.6, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-09-27", "l_receiptdate": "1996-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "hely regular deposits" }
+{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38966.55, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ts. pinto beans use according to th" }
+{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-14", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "onic epitaphs. f" }
+{ "l_orderkey": 4999, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31594.5, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-08-15", "l_receiptdate": "1993-08-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ades cajole carefully unusual ide" }
+{ "l_orderkey": 5184, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34753.95, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-10-16", "l_receiptdate": "1998-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully express asympto" }
+{ "l_orderkey": 5382, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-22", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular accounts. even accounts integrate" }
 { "l_orderkey": 5415, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11584.65, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-08-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts maintain carefully unusual" }
+{ "l_orderkey": 5767, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-02", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-06-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sits among the" }
 { "l_orderkey": 5856, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 41072.85, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uickly quickly fluffy in" }
 { "l_orderkey": 5859, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 36860.25, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-06-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular acco" }
 { "l_orderkey": 5958, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 44232.3, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1995-10-19", "l_receiptdate": "1996-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "n accounts. final, ironic packages " }
-{ "l_orderkey": 647, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15797.25, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ve the even, bold foxes sleep " }
-{ "l_orderkey": 1347, "l_partkey": 153, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 22116.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-10", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-11-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "g pinto beans affix car" }
-{ "l_orderkey": 1829, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14744.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-06-08", "l_receiptdate": "1994-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular deposits alongside of the flu" }
-{ "l_orderkey": 2276, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52657.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts dete" }
-{ "l_orderkey": 3910, "l_partkey": 153, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1053.15, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s sleep neve" }
-{ "l_orderkey": 4455, "l_partkey": 153, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49498.05, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-01", "l_commitdate": "1993-12-25", "l_receiptdate": "1994-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. even, even accou" }
-{ "l_orderkey": 4775, "l_partkey": 153, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35807.1, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-14", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "onic epitaphs. f" }
 { "l_orderkey": 7, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 39981.7, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns haggle carefully ironic deposits. bl" }
-{ "l_orderkey": 4454, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-06", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ully. carefully final accounts accordi" }
-{ "l_orderkey": 5159, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he furiously sile" }
+{ "l_orderkey": 422, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26303.75, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "carefully bold theodolit" }
+{ "l_orderkey": 449, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. blithely ironic " }
+{ "l_orderkey": 896, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11573.65, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "the multipliers sleep" }
 { "l_orderkey": 928, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 48398.9, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-04-09", "l_receiptdate": "1995-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " beans sleep against the carefully ir" }
+{ "l_orderkey": 962, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-06-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "across the furiously regular escapades daz" }
 { "l_orderkey": 1281, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13677.95, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully final platelets wa" }
+{ "l_orderkey": 1732, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9469.35, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-01-29", "l_receiptdate": "1994-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular platelets. deposits wak" }
 { "l_orderkey": 1889, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 43138.15, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-07-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s! furiously pending r" }
 { "l_orderkey": 1954, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32616.65, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "against the packages. bold, ironic e" }
 { "l_orderkey": 2050, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50503.2, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-10-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " final packages. pinto" }
-{ "l_orderkey": 2402, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25251.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-21", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "as; blithely ironic requ" }
-{ "l_orderkey": 3619, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 45242.45, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " bold, even" }
-{ "l_orderkey": 3841, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42086.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its. quickly regular ideas nag carefully" }
-{ "l_orderkey": 4327, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-28", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-05-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "arefully sile" }
-{ "l_orderkey": 4390, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 36825.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-30", "l_commitdate": "1995-07-02", "l_receiptdate": "1995-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ongside of the slyly regular ideas" }
-{ "l_orderkey": 4832, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1998-02-01", "l_receiptdate": "1998-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. blithely bold pinto beans should have" }
-{ "l_orderkey": 449, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-06", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. blithely ironic " }
-{ "l_orderkey": 962, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12625.8, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-06-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "across the furiously regular escapades daz" }
-{ "l_orderkey": 1732, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9469.35, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-01-29", "l_receiptdate": "1994-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular platelets. deposits wak" }
 { "l_orderkey": 2368, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16834.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-31", "l_commitdate": "1993-10-22", "l_receiptdate": "1993-11-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "telets wake carefully iro" }
+{ "l_orderkey": 2402, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 25251.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-21", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "as; blithely ironic requ" }
+{ "l_orderkey": 2403, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19990.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. ironic in" }
 { "l_orderkey": 3207, "l_partkey": 152, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17886.55, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eep against the instructions. gifts hag" }
+{ "l_orderkey": 3619, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 45242.45, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " bold, even" }
 { "l_orderkey": 3750, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 34720.95, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-27", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "theodolites haggle. slyly pendin" }
+{ "l_orderkey": 3841, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42086.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its. quickly regular ideas nag carefully" }
 { "l_orderkey": 3878, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 21043.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-13", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "about the carefully ironic pa" }
 { "l_orderkey": 4230, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18938.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " the final acco" }
+{ "l_orderkey": 4327, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-28", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-05-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "arefully sile" }
+{ "l_orderkey": 4390, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 36825.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-30", "l_commitdate": "1995-07-02", "l_receiptdate": "1995-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ongside of the slyly regular ideas" }
+{ "l_orderkey": 4454, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-06", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ully. carefully final accounts accordi" }
+{ "l_orderkey": 4832, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1998-02-01", "l_receiptdate": "1998-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. blithely bold pinto beans should have" }
 { "l_orderkey": 4871, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10521.5, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s integrate after the a" }
-{ "l_orderkey": 422, "l_partkey": 152, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26303.75, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-07-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "carefully bold theodolit" }
-{ "l_orderkey": 896, "l_partkey": 152, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11573.65, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-19", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "the multipliers sleep" }
-{ "l_orderkey": 2403, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19990.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. ironic in" }
+{ "l_orderkey": 5159, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23147.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he furiously sile" }
 { "l_orderkey": 5765, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48398.9, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ccounts sleep about th" }
-{ "l_orderkey": 5605, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-13", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lowly special courts nag among the furi" }
-{ "l_orderkey": 5730, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2102.3, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ely ironic foxes. carefu" }
 { "l_orderkey": 224, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16818.4, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y unusual foxes " }
 { "l_orderkey": 519, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3153.45, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1998-01-25", "l_receiptdate": "1998-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "erve blithely blithely ironic asymp" }
 { "l_orderkey": 579, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9460.35, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-20", "l_commitdate": "1998-04-28", "l_receiptdate": "1998-07-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e ironic, express deposits are furiously" }
-{ "l_orderkey": 1061, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es are slyly expr" }
-{ "l_orderkey": 3014, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 50455.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1993-01-08", "l_receiptdate": "1992-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y pending theodolites wake. reg" }
-{ "l_orderkey": 3969, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22074.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-16", "l_receiptdate": "1997-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts doze quickly final reque" }
-{ "l_orderkey": 4454, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21023.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lar theodolites. even instructio" }
-{ "l_orderkey": 5537, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-11-08", "l_receiptdate": "1997-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly bold packages are. qu" }
 { "l_orderkey": 773, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quickly eve" }
-{ "l_orderkey": 1122, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15767.25, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-15", "l_commitdate": "1997-03-15", "l_receiptdate": "1997-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olve blithely regular, " }
-{ "l_orderkey": 4931, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8409.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts boost. packages wake sly" }
-{ "l_orderkey": 5093, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32585.65, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-09-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " against the" }
-{ "l_orderkey": 5222, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1051.15, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-19", "l_commitdate": "1994-07-16", "l_receiptdate": "1994-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "idle requests. carefully pending pinto bean" }
+{ "l_orderkey": 1061, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "es are slyly expr" }
 { "l_orderkey": 1089, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49404.05, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-26", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "aggle furiously among the bravely eve" }
+{ "l_orderkey": 1122, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15767.25, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-15", "l_commitdate": "1997-03-15", "l_receiptdate": "1997-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olve blithely regular, " }
 { "l_orderkey": 1248, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38892.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-26", "l_commitdate": "1992-02-05", "l_receiptdate": "1992-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": ". final requests integrate quickly. blit" }
 { "l_orderkey": 1664, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10511.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-10", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-05-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "instructions up the acc" }
 { "l_orderkey": 2727, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3153.45, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the carefully regular foxes u" }
 { "l_orderkey": 2822, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-11", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-09-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kly about the sly" }
+{ "l_orderkey": 3014, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 50455.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1993-01-08", "l_receiptdate": "1992-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y pending theodolites wake. reg" }
 { "l_orderkey": 3365, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38892.55, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "requests. quickly pending instructions a" }
 { "l_orderkey": 3462, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4204.6, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ackages. fu" }
 { "l_orderkey": 3587, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37841.4, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ully regular excuse" }
+{ "l_orderkey": 3969, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 22074.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-16", "l_receiptdate": "1997-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts doze quickly final reque" }
 { "l_orderkey": 4256, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 23125.3, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ", final platelets are slyly final pint" }
-{ "l_orderkey": 2499, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15752.25, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-12-06", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " slyly across the slyly" }
-{ "l_orderkey": 4773, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely final deposits nag after t" }
-{ "l_orderkey": 5157, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27303.9, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nto beans cajole car" }
-{ "l_orderkey": 5892, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38855.55, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-12", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "maintain. bold, expre" }
+{ "l_orderkey": 4454, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 21023.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lar theodolites. even instructio" }
+{ "l_orderkey": 4931, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8409.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-01-14", "l_receiptdate": "1995-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts boost. packages wake sly" }
+{ "l_orderkey": 5093, "l_partkey": 151, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32585.65, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-09-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " against the" }
+{ "l_orderkey": 5222, "l_partkey": 151, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1051.15, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-19", "l_commitdate": "1994-07-16", "l_receiptdate": "1994-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "idle requests. carefully pending pinto bean" }
+{ "l_orderkey": 5537, "l_partkey": 151, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40994.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-11-08", "l_receiptdate": "1997-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly bold packages are. qu" }
+{ "l_orderkey": 5605, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7358.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-13", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lowly special courts nag among the furi" }
+{ "l_orderkey": 5730, "l_partkey": 151, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2102.3, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ely ironic foxes. carefu" }
 { "l_orderkey": 485, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52507.5, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-28", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "iously quick excuses. carefully final f" }
 { "l_orderkey": 677, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26253.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-12", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " packages integrate blithely" }
-{ "l_orderkey": 2532, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-26", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "er the slyly pending" }
-{ "l_orderkey": 4805, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7351.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " requests. regular deposit" }
-{ "l_orderkey": 4864, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29404.2, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "thely around the bli" }
 { "l_orderkey": 1025, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37805.4, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e unusual, regular instr" }
 { "l_orderkey": 1251, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7351.05, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously pe" }
 { "l_orderkey": 1317, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37805.4, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-03", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits. quic" }
 { "l_orderkey": 1701, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49357.05, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-06-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly final requests cajole requests. f" }
-{ "l_orderkey": 3333, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28354.05, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-06", "l_commitdate": "1992-10-26", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s dazzle fluffil" }
-{ "l_orderkey": 4550, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9451.35, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l dependencies boost slyly after th" }
-{ "l_orderkey": 5319, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32554.65, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-26", "l_commitdate": "1996-03-07", "l_receiptdate": "1996-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "d carefully about the courts. fluffily spe" }
 { "l_orderkey": 1829, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12601.8, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-23", "l_commitdate": "1994-07-13", "l_receiptdate": "1994-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ges wake furiously express pinto" }
 { "l_orderkey": 1856, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 23103.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-05-26", "l_receiptdate": "1992-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets detect slyly regular packages. ca" }
+{ "l_orderkey": 2499, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15752.25, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-12-06", "l_receiptdate": "1996-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " slyly across the slyly" }
+{ "l_orderkey": 2532, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-26", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "er the slyly pending" }
 { "l_orderkey": 2561, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2100.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-14", "l_commitdate": "1998-01-21", "l_receiptdate": "1998-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s are. silently silent foxes sleep about" }
+{ "l_orderkey": 3333, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28354.05, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-06", "l_commitdate": "1992-10-26", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s dazzle fluffil" }
 { "l_orderkey": 4038, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30454.35, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-07", "l_commitdate": "1996-03-08", "l_receiptdate": "1996-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ffix. quietly ironic packages a" }
 { "l_orderkey": 4192, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 46206.6, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "structions mai" }
 { "l_orderkey": 4423, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3150.45, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-22", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-04-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final theodolites nag after the bli" }
+{ "l_orderkey": 4550, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9451.35, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-02-07", "l_receiptdate": "1995-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l dependencies boost slyly after th" }
 { "l_orderkey": 4674, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 52507.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-13", "l_commitdate": "1994-06-15", "l_receiptdate": "1994-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "haggle about the blithel" }
 { "l_orderkey": 4711, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 23103.3, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-06-18", "l_receiptdate": "1998-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "along the quickly careful packages. bli" }
+{ "l_orderkey": 4773, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 21003.0, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely final deposits nag after t" }
+{ "l_orderkey": 4805, "l_partkey": 150, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7351.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " requests. regular deposit" }
+{ "l_orderkey": 4864, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29404.2, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "thely around the bli" }
 { "l_orderkey": 4931, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26253.75, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-01-05", "l_receiptdate": "1994-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "aggle bravely according to the quic" }
+{ "l_orderkey": 5157, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27303.9, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nto beans cajole car" }
 { "l_orderkey": 5253, "l_partkey": 150, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39905.7, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "onic dependencies are furiou" }
+{ "l_orderkey": 5319, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32554.65, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-26", "l_commitdate": "1996-03-07", "l_receiptdate": "1996-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "d carefully about the courts. fluffily spe" }
 { "l_orderkey": 5444, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 42006.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even packages." }
 { "l_orderkey": 5537, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15752.25, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits. permanently pending packag" }
 { "l_orderkey": 5734, "l_partkey": 150, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6300.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-27", "l_commitdate": "1997-12-19", "l_receiptdate": "1997-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s. regular platelets cajole furiously. regu" }
-{ "l_orderkey": 387, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33572.48, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gle. silent, fur" }
-{ "l_orderkey": 1380, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6294.84, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e foxes. slyly specia" }
-{ "l_orderkey": 2306, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "f the slyly unusual accounts. furiousl" }
-{ "l_orderkey": 4514, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12589.68, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " carefully ironic foxes nag caref" }
-{ "l_orderkey": 4832, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1998-02-12", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ages. slyly express deposits cajole car" }
-{ "l_orderkey": 5346, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22031.94, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-11", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "integrate blithely a" }
-{ "l_orderkey": 5798, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7343.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-06-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts against the blithely final p" }
+{ "l_orderkey": 5892, "l_partkey": 150, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38855.55, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-12", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "maintain. bold, expre" }
 { "l_orderkey": 194, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16786.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-05-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y regular requests. furious" }
+{ "l_orderkey": 198, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15737.1, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. quickly pending deposits s" }
+{ "l_orderkey": 387, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33572.48, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gle. silent, fur" }
 { "l_orderkey": 419, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 17835.38, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-20", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar dependencies: carefully regu" }
-{ "l_orderkey": 1638, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31474.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s cajole boldly bold requests. closely " }
-{ "l_orderkey": 2438, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely; blithely special pinto beans breach" }
-{ "l_orderkey": 3298, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-05-24", "l_receiptdate": "1996-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly final accou" }
-{ "l_orderkey": 4960, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "e blithely carefully fina" }
+{ "l_orderkey": 1380, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6294.84, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e foxes. slyly specia" }
 { "l_orderkey": 1415, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 26228.5, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-07-12", "l_receiptdate": "1994-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ect never fluff" }
+{ "l_orderkey": 1479, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34621.62, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully special courts affix. fluff" }
+{ "l_orderkey": 1638, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31474.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s cajole boldly bold requests. closely " }
 { "l_orderkey": 1671, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22031.94, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-28", "l_commitdate": "1996-09-28", "l_receiptdate": "1996-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s accounts slee" }
 { "l_orderkey": 2081, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13638.82, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fter the even deposi" }
+{ "l_orderkey": 2306, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "f the slyly unusual accounts. furiousl" }
 { "l_orderkey": 2368, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-03", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ng the doggedly ironic requests are blithe" }
+{ "l_orderkey": 2438, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely; blithely special pinto beans breach" }
 { "l_orderkey": 2688, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 44063.88, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-04-04", "l_receiptdate": "1992-05-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lly even account" }
 { "l_orderkey": 2724, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30425.06, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-10", "l_commitdate": "1994-11-17", "l_receiptdate": "1995-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "l requests hagg" }
+{ "l_orderkey": 2754, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-05-15", "l_receiptdate": "1994-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely silent requests. regular depo" }
 { "l_orderkey": 3107, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16786.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-10-20", "l_receiptdate": "1997-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular pinto beans. ironic ideas haggle" }
 { "l_orderkey": 3296, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32523.34, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-26", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ainst the furi" }
+{ "l_orderkey": 3298, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-05-24", "l_receiptdate": "1996-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly final accou" }
+{ "l_orderkey": 3300, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he fluffily final a" }
+{ "l_orderkey": 4514, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12589.68, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-09-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " carefully ironic foxes nag caref" }
+{ "l_orderkey": 4546, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10491.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-16", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "above the enticingly ironic dependencies" }
+{ "l_orderkey": 4832, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1998-02-12", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ages. slyly express deposits cajole car" }
 { "l_orderkey": 4871, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 36719.9, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-11", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ackages sle" }
+{ "l_orderkey": 4928, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35670.76, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-12", "l_commitdate": "1993-12-31", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", regular depos" }
+{ "l_orderkey": 4960, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9442.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "e blithely carefully fina" }
+{ "l_orderkey": 5157, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41965.6, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-08-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ial packages according to " }
+{ "l_orderkey": 5346, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 22031.94, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-11", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "integrate blithely a" }
+{ "l_orderkey": 5382, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3147.42, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully unusua" }
 { "l_orderkey": 5445, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10491.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-10-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ncies abou" }
 { "l_orderkey": 5766, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40916.46, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-12-07", "l_receiptdate": "1993-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " furiously unusual courts. slyly final pear" }
+{ "l_orderkey": 5798, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7343.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-06-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts against the blithely final p" }
 { "l_orderkey": 5958, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34621.62, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lar, regular accounts wake furi" }
-{ "l_orderkey": 198, "l_partkey": 149, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15737.1, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-02-26", "l_receiptdate": "1998-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. quickly pending deposits s" }
-{ "l_orderkey": 1479, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34621.62, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully special courts affix. fluff" }
-{ "l_orderkey": 2754, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4196.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-05-15", "l_receiptdate": "1994-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely silent requests. regular depo" }
-{ "l_orderkey": 3300, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 24130.22, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-09-03", "l_receiptdate": "1995-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he fluffily final a" }
-{ "l_orderkey": 4546, "l_partkey": 149, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10491.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-09-16", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "above the enticingly ironic dependencies" }
-{ "l_orderkey": 4928, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35670.76, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-12", "l_commitdate": "1993-12-31", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", regular depos" }
-{ "l_orderkey": 5157, "l_partkey": 149, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41965.6, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-08-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ial packages according to " }
-{ "l_orderkey": 5382, "l_partkey": 149, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3147.42, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully unusua" }
+{ "l_orderkey": 164, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-26", "l_commitdate": "1993-01-03", "l_receiptdate": "1992-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y carefully regular dep" }
+{ "l_orderkey": 353, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 30396.06, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions impr" }
+{ "l_orderkey": 515, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39829.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-19", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ays. furiously express requests haggle furi" }
+{ "l_orderkey": 677, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1994-01-14", "l_receiptdate": "1993-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. regular " }
+{ "l_orderkey": 774, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35636.76, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar excuses are furiously final instr" }
+{ "l_orderkey": 967, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the slyly even ideas. carefully even" }
+{ "l_orderkey": 1154, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 52407.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ove the furiously bold Tires" }
 { "l_orderkey": 1408, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30396.06, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-02-14", "l_receiptdate": "1998-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "en accounts grow furiousl" }
 { "l_orderkey": 1408, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20962.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-01-25", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " blithely fluffi" }
 { "l_orderkey": 1508, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s the blithely bold instruction" }
+{ "l_orderkey": 1632, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14673.96, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-15", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-01-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oxes. deposits nag slyly along the slyly " }
 { "l_orderkey": 1825, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40877.46, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ual, bold ideas haggle above the quickly ir" }
+{ "l_orderkey": 1893, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes bo" }
+{ "l_orderkey": 2465, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47166.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y silent foxes. final pinto beans above " }
 { "l_orderkey": 2531, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-27", "l_commitdate": "1996-07-03", "l_receiptdate": "1996-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t the dogged, un" }
 { "l_orderkey": 2562, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-10-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " slyly final ideas haggle car" }
-{ "l_orderkey": 2851, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-12", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-12-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y special theodolites. carefully" }
-{ "l_orderkey": 3172, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " final packages. " }
-{ "l_orderkey": 4934, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-04-09", "l_receiptdate": "1997-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle alongside of the" }
-{ "l_orderkey": 4964, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 48214.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "althy deposits" }
-{ "l_orderkey": 164, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-26", "l_commitdate": "1993-01-03", "l_receiptdate": "1992-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y carefully regular dep" }
-{ "l_orderkey": 677, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1048.14, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1994-01-14", "l_receiptdate": "1993-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly. regular " }
-{ "l_orderkey": 774, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35636.76, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lar excuses are furiously final instr" }
-{ "l_orderkey": 1154, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 52407.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-05-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ove the furiously bold Tires" }
-{ "l_orderkey": 2465, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 47166.3, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y silent foxes. final pinto beans above " }
 { "l_orderkey": 2562, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 38781.18, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-29", "l_commitdate": "1992-10-06", "l_receiptdate": "1992-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ". slyly regular ideas according to the fl" }
-{ "l_orderkey": 3877, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely about the dogged ideas. ac" }
-{ "l_orderkey": 3908, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r instructions was requests. ironically " }
-{ "l_orderkey": 4995, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-01", "l_receiptdate": "1996-04-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "t blithely. requests affix blithely. " }
-{ "l_orderkey": 5156, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly even orbi" }
-{ "l_orderkey": 515, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39829.32, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-19", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ays. furiously express requests haggle furi" }
+{ "l_orderkey": 2566, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19914.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-11-24", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests. silent" }
 { "l_orderkey": 2598, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41925.6, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "the enticing" }
+{ "l_orderkey": 2753, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 20962.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " express pack" }
 { "l_orderkey": 2757, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27251.64, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-10-02", "l_receiptdate": "1995-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "around the blithely" }
 { "l_orderkey": 2790, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11529.54, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-28", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "lar requests poach slyly foxes" }
+{ "l_orderkey": 2851, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-12", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-12-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y special theodolites. carefully" }
 { "l_orderkey": 3111, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42973.74, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-12-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily slow ideas. " }
-{ "l_orderkey": 4547, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15722.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic gifts integrate " }
-{ "l_orderkey": 5188, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "r attainments are across the " }
-{ "l_orderkey": 5601, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12577.68, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ep carefully a" }
-{ "l_orderkey": 5793, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly enticing excuses use slyly abov" }
-{ "l_orderkey": 353, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 30396.06, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions impr" }
-{ "l_orderkey": 967, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the slyly even ideas. carefully even" }
-{ "l_orderkey": 1632, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14673.96, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-15", "l_commitdate": "1997-02-25", "l_receiptdate": "1997-01-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oxes. deposits nag slyly along the slyly " }
-{ "l_orderkey": 1893, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51358.86, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y final foxes bo" }
-{ "l_orderkey": 2566, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19914.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-11-24", "l_receiptdate": "1992-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests. silent" }
-{ "l_orderkey": 2753, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 20962.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " express pack" }
+{ "l_orderkey": 3172, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 45070.02, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " final packages. " }
 { "l_orderkey": 3425, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 25155.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ajole blithely sl" }
 { "l_orderkey": 3712, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39829.32, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-15", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s nag carefully-- even, reg" }
 { "l_orderkey": 3840, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42973.74, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " nag slyly? slyly pending accounts " }
+{ "l_orderkey": 3877, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely about the dogged ideas. ac" }
+{ "l_orderkey": 3908, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r instructions was requests. ironically " }
 { "l_orderkey": 4161, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19914.66, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-11-11", "l_receiptdate": "1993-09-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "beans breach s" }
+{ "l_orderkey": 4547, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15722.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic gifts integrate " }
 { "l_orderkey": 4838, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2096.28, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-09-16", "l_receiptdate": "1992-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "hely final notornis are furiously blithe" }
+{ "l_orderkey": 4934, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-04-09", "l_receiptdate": "1997-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle alongside of the" }
+{ "l_orderkey": 4964, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 48214.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "althy deposits" }
+{ "l_orderkey": 4995, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-01", "l_receiptdate": "1996-04-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "t blithely. requests affix blithely. " }
+{ "l_orderkey": 5156, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 37733.04, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly even orbi" }
+{ "l_orderkey": 5188, "l_partkey": 148, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9433.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "r attainments are across the " }
+{ "l_orderkey": 5601, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12577.68, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ep carefully a" }
 { "l_orderkey": 5760, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8385.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l accounts among the carefully even de" }
+{ "l_orderkey": 5793, "l_partkey": 148, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50310.72, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly enticing excuses use slyly abov" }
 { "l_orderkey": 5858, "l_partkey": 148, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7336.98, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". doggedly regular packages use pendin" }
 { "l_orderkey": 5892, "l_partkey": 148, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7336.98, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-26", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously. quickly even deposits da" }
-{ "l_orderkey": 2016, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2094.28, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "carefully according to the " }
-{ "l_orderkey": 2117, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3141.42, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tes cajole" }
-{ "l_orderkey": 2658, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 28272.78, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-09-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ecial packages use abov" }
-{ "l_orderkey": 2724, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21989.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-15", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "as. carefully regular dependencies wak" }
-{ "l_orderkey": 614, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14659.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-14", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ular packages haggle about the pack" }
-{ "l_orderkey": 2279, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12565.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccounts. slyl" }
-{ "l_orderkey": 4227, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 51309.86, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-19", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts sleep blithely carefully unusual ideas." }
-{ "l_orderkey": 4992, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49215.58, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-04", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "atterns use fluffily." }
-{ "l_orderkey": 5380, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10471.4, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1998-01-10", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully pending deposits. special, even t" }
-{ "l_orderkey": 5569, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19895.66, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-30", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " detect ca" }
-{ "l_orderkey": 5959, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17801.38, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. blithely ex" }
 { "l_orderkey": 225, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25131.36, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-10-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ironic accounts are final account" }
+{ "l_orderkey": 257, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7329.98, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ackages sleep bold realms. f" }
 { "l_orderkey": 258, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 37697.04, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nic asymptotes. slyly silent r" }
+{ "l_orderkey": 614, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14659.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-14", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ular packages haggle about the pack" }
 { "l_orderkey": 931, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 50262.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep alongside of the fluffy " }
 { "l_orderkey": 1122, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 26178.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "d furiously. pinto " }
-{ "l_orderkey": 1155, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24084.22, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly unusual packages. iro" }
-{ "l_orderkey": 1600, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31414.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-03", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al escapades alongside of the depo" }
-{ "l_orderkey": 3073, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40838.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar excuses across the furiously even " }
-{ "l_orderkey": 4610, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30367.06, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " foxes. special, express package" }
-{ "l_orderkey": 5954, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8377.12, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unusual th" }
-{ "l_orderkey": 257, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7329.98, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ackages sleep bold realms. f" }
 { "l_orderkey": 1126, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 14659.96, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nstructions. blithe" }
+{ "l_orderkey": 1155, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 24084.22, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly unusual packages. iro" }
 { "l_orderkey": 1184, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4188.56, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " express packages. slyly expres" }
+{ "l_orderkey": 1600, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31414.2, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-03", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-06-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al escapades alongside of the depo" }
 { "l_orderkey": 1763, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13612.82, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1997-01-24", "l_receiptdate": "1996-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s sleep carefully. fluffily unusua" }
+{ "l_orderkey": 2016, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2094.28, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-11-09", "l_receiptdate": "1996-10-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "carefully according to the " }
+{ "l_orderkey": 2117, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3141.42, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tes cajole" }
+{ "l_orderkey": 2279, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12565.68, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccounts. slyl" }
 { "l_orderkey": 2404, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37697.04, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-05-16", "l_receiptdate": "1997-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s nag furi" }
+{ "l_orderkey": 2658, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 28272.78, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-09-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ecial packages use abov" }
+{ "l_orderkey": 2724, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21989.94, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-15", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "as. carefully regular dependencies wak" }
+{ "l_orderkey": 3073, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40838.46, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar excuses across the furiously even " }
 { "l_orderkey": 3492, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31414.2, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-29", "l_commitdate": "1995-01-02", "l_receiptdate": "1995-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " detect furiously permanent, unusual accou" }
 { "l_orderkey": 3748, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 21989.94, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fix carefully furiously express ideas. furi" }
+{ "l_orderkey": 4227, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 51309.86, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-19", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-06-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts sleep blithely carefully unusual ideas." }
 { "l_orderkey": 4321, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34555.62, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "yly special excuses. fluffily " }
 { "l_orderkey": 4453, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 42932.74, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "anent theodolites are slyly except t" }
+{ "l_orderkey": 4610, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30367.06, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " foxes. special, express package" }
 { "l_orderkey": 4647, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 28272.78, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ully even ti" }
+{ "l_orderkey": 4992, "l_partkey": 147, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49215.58, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-04", "l_commitdate": "1992-08-05", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "atterns use fluffily." }
+{ "l_orderkey": 5380, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10471.4, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1998-01-10", "l_receiptdate": "1997-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully pending deposits. special, even t" }
 { "l_orderkey": 5543, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8377.12, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-11-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "totes? iron" }
+{ "l_orderkey": 5569, "l_partkey": 147, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19895.66, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-30", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " detect ca" }
+{ "l_orderkey": 5954, "l_partkey": 147, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8377.12, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unusual th" }
+{ "l_orderkey": 5959, "l_partkey": 147, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17801.38, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. blithely ex" }
+{ "l_orderkey": 7, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. instructions" }
 { "l_orderkey": 194, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 37661.04, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "pecial packages wake after the slyly r" }
+{ "l_orderkey": 610, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40799.46, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "counts. ironic warhorses are " }
 { "l_orderkey": 678, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20922.8, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "furiously express excuses. foxes eat fu" }
+{ "l_orderkey": 711, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2092.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely across t" }
 { "l_orderkey": 993, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 34522.62, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-10-24", "l_receiptdate": "1995-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fluffily. quiet excuses sleep furiously sly" }
 { "l_orderkey": 1286, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 42891.74, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-02", "l_commitdate": "1993-08-06", "l_receiptdate": "1993-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " the furiously expre" }
-{ "l_orderkey": 2406, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35568.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hinly even accounts are slyly q" }
-{ "l_orderkey": 3041, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-08-14", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "iously across the silent pinto beans. furi" }
-{ "l_orderkey": 5285, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1046.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing deposits integra" }
-{ "l_orderkey": 5798, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 41845.6, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " integrate carefu" }
-{ "l_orderkey": 7, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. instructions" }
+{ "l_orderkey": 1733, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13599.82, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-03", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-08-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites sleep furious" }
 { "l_orderkey": 2215, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20922.8, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " unusual deposits haggle carefully. ide" }
+{ "l_orderkey": 2406, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35568.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hinly even accounts are slyly q" }
 { "l_orderkey": 2722, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15692.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-02", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully final asympt" }
+{ "l_orderkey": 2884, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26153.5, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1997-12-06", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "onic theodolites with the instructi" }
+{ "l_orderkey": 3041, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9415.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-08-14", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "iously across the silent pinto beans. furi" }
 { "l_orderkey": 3394, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44984.02, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "hockey players. slyly regular requests afte" }
 { "l_orderkey": 3584, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11507.54, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lithely slyly " }
-{ "l_orderkey": 4772, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16738.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "egular accounts wake s" }
-{ "l_orderkey": 5895, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32430.34, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final deposits nod slyly careful" }
-{ "l_orderkey": 610, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40799.46, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "counts. ironic warhorses are " }
-{ "l_orderkey": 1733, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13599.82, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-03", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-08-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites sleep furious" }
 { "l_orderkey": 3714, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14645.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ending ideas. thinly unusual theodo" }
+{ "l_orderkey": 4198, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50214.72, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-09-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole carefully final, ironic ide" }
+{ "l_orderkey": 4676, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4184.56, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "detect above the ironic platelets. fluffily" }
+{ "l_orderkey": 4772, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16738.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-07", "l_receiptdate": "1994-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "egular accounts wake s" }
 { "l_orderkey": 4960, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 38707.18, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ending theodolites w" }
 { "l_orderkey": 5185, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 52307.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-19", "l_receiptdate": "1997-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "final platelets. ideas sleep careful" }
 { "l_orderkey": 5249, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 30338.06, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-16", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " players. f" }
-{ "l_orderkey": 711, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2092.28, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-01", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely across t" }
-{ "l_orderkey": 2884, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26153.5, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1997-12-06", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "onic theodolites with the instructi" }
-{ "l_orderkey": 4198, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50214.72, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-03", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-09-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole carefully final, ironic ide" }
-{ "l_orderkey": 4676, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4184.56, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-12", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "detect above the ironic platelets. fluffily" }
+{ "l_orderkey": 5285, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1046.14, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-04-02", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing deposits integra" }
 { "l_orderkey": 5345, "l_partkey": 146, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2092.28, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-18", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the slyly specia" }
+{ "l_orderkey": 5798, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 41845.6, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " integrate carefu" }
+{ "l_orderkey": 5895, "l_partkey": 146, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32430.34, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final deposits nod slyly careful" }
 { "l_orderkey": 5921, "l_partkey": 146, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 26153.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-19", "l_commitdate": "1994-06-15", "l_receiptdate": "1994-06-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nd the slyly regular deposits. quick" }
-{ "l_orderkey": 3812, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34489.62, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-10", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "posits engage. ironic, regular p" }
-{ "l_orderkey": 4327, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-27", "l_receiptdate": "1995-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic dolphins" }
-{ "l_orderkey": 4998, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 25083.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-25", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " unwind about" }
-{ "l_orderkey": 1797, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16722.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-06-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans wake regular accounts. blit" }
-{ "l_orderkey": 2531, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48076.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-27", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e final, bold pains. ir" }
-{ "l_orderkey": 3873, "l_partkey": 145, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45986.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly even platelets wake. " }
-{ "l_orderkey": 4198, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13586.82, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-18", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furious excuses. bli" }
-{ "l_orderkey": 930, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10451.4, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-02-17", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "blithely bold i" }
-{ "l_orderkey": 2118, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1996-12-20", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y ironic accounts sleep upon the packages. " }
-{ "l_orderkey": 3554, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18812.52, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle. furiously fluffy requests ac" }
-{ "l_orderkey": 3687, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33444.48, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deas cajole fo" }
-{ "l_orderkey": 3907, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 42850.74, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-25", "l_commitdate": "1992-10-17", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s above the unusual ideas sleep furiousl" }
-{ "l_orderkey": 4711, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15677.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " beans wake. deposits could bo" }
-{ "l_orderkey": 5954, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20902.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-03-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ke furiously blithely special packa" }
 { "l_orderkey": 134, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 49121.58, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-16", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s! carefully unusual requests boost careful" }
 { "l_orderkey": 583, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1045.14, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " regular, regular ideas. even, bra" }
 { "l_orderkey": 834, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37625.04, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-28", "l_commitdate": "1994-07-25", "l_receiptdate": "1994-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ccounts haggle after the furiously " }
+{ "l_orderkey": 930, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10451.4, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-02-17", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "blithely bold i" }
 { "l_orderkey": 1762, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 25083.36, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts solve alongside of the fluffily " }
+{ "l_orderkey": 1797, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16722.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-06-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans wake regular accounts. blit" }
+{ "l_orderkey": 2118, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1996-12-20", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y ironic accounts sleep upon the packages. " }
+{ "l_orderkey": 2531, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48076.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-27", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e final, bold pains. ir" }
 { "l_orderkey": 3109, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 51211.86, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even pearls. furiously pending " }
+{ "l_orderkey": 3554, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18812.52, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle. furiously fluffy requests ac" }
 { "l_orderkey": 3653, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39715.32, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-05-13", "l_receiptdate": "1994-07-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ainst the " }
+{ "l_orderkey": 3687, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33444.48, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-07", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deas cajole fo" }
+{ "l_orderkey": 3812, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34489.62, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-10", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "posits engage. ironic, regular p" }
+{ "l_orderkey": 3873, "l_partkey": 145, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45986.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly even platelets wake. " }
 { "l_orderkey": 3877, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 49121.58, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-08-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "furiously quick requests nag along the theo" }
+{ "l_orderkey": 3907, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 42850.74, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-25", "l_commitdate": "1992-10-17", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s above the unusual ideas sleep furiousl" }
+{ "l_orderkey": 4198, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13586.82, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-18", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-08-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furious excuses. bli" }
+{ "l_orderkey": 4327, "l_partkey": 145, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11496.54, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-27", "l_receiptdate": "1995-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic dolphins" }
 { "l_orderkey": 4512, "l_partkey": 145, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 21947.94, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-12-30", "l_receiptdate": "1995-11-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lly unusual pinto b" }
+{ "l_orderkey": 4711, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15677.1, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " beans wake. deposits could bo" }
 { "l_orderkey": 4807, "l_partkey": 145, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35534.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas. deposits according to the fin" }
+{ "l_orderkey": 4998, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 25083.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-25", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " unwind about" }
 { "l_orderkey": 5667, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38670.18, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-09-17", "l_receiptdate": "1995-10-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s cajole blit" }
+{ "l_orderkey": 5954, "l_partkey": 145, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20902.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-27", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-03-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ke furiously blithely special packa" }
 { "l_orderkey": 327, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-05", "l_commitdate": "1995-06-07", "l_receiptdate": "1995-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "cial ideas sleep af" }
-{ "l_orderkey": 1475, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31324.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1997-12-30", "l_receiptdate": "1998-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular theodolites mold across th" }
-{ "l_orderkey": 2594, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48030.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "beans. instructions across t" }
-{ "l_orderkey": 3746, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29235.92, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s after the even, special requests" }
-{ "l_orderkey": 4485, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 44898.02, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-28", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". blithely" }
-{ "l_orderkey": 4992, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17750.38, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s along the perma" }
-{ "l_orderkey": 1441, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "egular courts. fluffily even grouches " }
-{ "l_orderkey": 2565, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43853.88, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ngly silent " }
-{ "l_orderkey": 2789, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "d the carefully iron" }
-{ "l_orderkey": 3361, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6264.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages sleep. furiously unus" }
-{ "l_orderkey": 3648, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-14", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s nag packages." }
-{ "l_orderkey": 4518, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9397.26, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-07-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " pending deposits. slyly re" }
-{ "l_orderkey": 5154, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15662.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-23", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "even packages. packages use" }
-{ "l_orderkey": 5441, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34456.62, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-10-06", "l_receiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ges. final instruction" }
-{ "l_orderkey": 5826, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4176.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages across the fluffily spec" }
-{ "l_orderkey": 2439, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ites. furiously" }
-{ "l_orderkey": 2497, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50118.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even, regular requests across " }
-{ "l_orderkey": 2695, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 21926.94, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. furiously ironic platelets ar" }
-{ "l_orderkey": 3521, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "onic dependencies haggle. fur" }
-{ "l_orderkey": 4065, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11485.54, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hang silently about " }
-{ "l_orderkey": 4675, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12529.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits affix carefully" }
-{ "l_orderkey": 4996, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13573.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-17", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "o beans use about the furious" }
-{ "l_orderkey": 5442, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "have to sleep furiously bold ideas. blith" }
 { "l_orderkey": 1061, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 36544.9, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-05", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ending requests nag careful" }
 { "l_orderkey": 1381, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 49074.58, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-22", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic deposits" }
+{ "l_orderkey": 1441, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "egular courts. fluffily even grouches " }
+{ "l_orderkey": 1475, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31324.2, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1997-12-30", "l_receiptdate": "1998-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular theodolites mold across th" }
+{ "l_orderkey": 2439, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5220.7, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ites. furiously" }
+{ "l_orderkey": 2497, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 50118.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even, regular requests across " }
+{ "l_orderkey": 2565, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43853.88, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ngly silent " }
+{ "l_orderkey": 2594, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 48030.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "beans. instructions across t" }
+{ "l_orderkey": 2695, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 21926.94, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. furiously ironic platelets ar" }
+{ "l_orderkey": 2789, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "d the carefully iron" }
 { "l_orderkey": 3203, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24015.22, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uses. fluffily ironic pinto bea" }
+{ "l_orderkey": 3361, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6264.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages sleep. furiously unus" }
 { "l_orderkey": 3457, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 46986.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-12", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages. care" }
+{ "l_orderkey": 3521, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "onic dependencies haggle. fur" }
 { "l_orderkey": 3618, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 50118.72, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tructions atop the ironi" }
+{ "l_orderkey": 3648, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16706.24, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-14", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s nag packages." }
+{ "l_orderkey": 3746, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29235.92, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s after the even, special requests" }
+{ "l_orderkey": 4065, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11485.54, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hang silently about " }
+{ "l_orderkey": 4485, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 44898.02, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-28", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". blithely" }
+{ "l_orderkey": 4518, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9397.26, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-07-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " pending deposits. slyly re" }
+{ "l_orderkey": 4675, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12529.68, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-22", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits affix carefully" }
 { "l_orderkey": 4773, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 24015.22, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-01", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-01-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly express grouches wak" }
 { "l_orderkey": 4931, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20882.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-25", "l_commitdate": "1994-12-21", "l_receiptdate": "1995-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furious" }
 { "l_orderkey": 4966, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12529.68, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "eodolites. ironic requests across the exp" }
+{ "l_orderkey": 4992, "l_partkey": 144, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17750.38, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s along the perma" }
+{ "l_orderkey": 4996, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13573.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-17", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "o beans use about the furious" }
+{ "l_orderkey": 5154, "l_partkey": 144, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15662.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-23", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "even packages. packages use" }
 { "l_orderkey": 5415, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 48030.44, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the fluffily " }
+{ "l_orderkey": 5441, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 34456.62, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-10-06", "l_receiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ges. final instruction" }
+{ "l_orderkey": 5442, "l_partkey": 144, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27147.64, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-03-21", "l_receiptdate": "1998-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "have to sleep furiously bold ideas. blith" }
+{ "l_orderkey": 5826, "l_partkey": 144, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4176.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages across the fluffily spec" }
+{ "l_orderkey": 263, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52157.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "re the packages. special" }
 { "l_orderkey": 323, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9388.26, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nic accounts. regular, regular pack" }
 { "l_orderkey": 450, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 33380.48, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-02", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " accounts nod fluffily even, pending" }
-{ "l_orderkey": 1285, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 46941.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-05", "l_commitdate": "1992-08-08", "l_receiptdate": "1992-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special requests haggle blithely." }
-{ "l_orderkey": 1542, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21905.94, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-13", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending foxes nag blithely " }
-{ "l_orderkey": 1638, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26078.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gle final, ironic pinto beans. " }
-{ "l_orderkey": 1763, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3129.42, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ut the slyly pending deposi" }
-{ "l_orderkey": 2913, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle. even, bold instructi" }
-{ "l_orderkey": 3458, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37553.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s lose. blithely ironic requests boost" }
-{ "l_orderkey": 4198, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47984.44, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits among th" }
-{ "l_orderkey": 2498, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50070.72, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-25", "l_commitdate": "1994-01-09", "l_receiptdate": "1993-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "onic requests wake" }
-{ "l_orderkey": 3553, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4172.56, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-13", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "olites boost bli" }
-{ "l_orderkey": 4834, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39639.32, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "alongside of the carefully even plate" }
-{ "l_orderkey": 5348, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14603.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "en pinto beans. somas cajo" }
-{ "l_orderkey": 5729, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1994-11-11", "l_receiptdate": "1994-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. even sheaves nag courts. " }
-{ "l_orderkey": 5921, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 42768.74, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual, regular theodol" }
 { "l_orderkey": 672, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 36509.9, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-07-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " dependencies haggle quickly. theo" }
 { "l_orderkey": 676, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11474.54, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-09", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "he final acco" }
 { "l_orderkey": 678, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16690.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "equests cajole around the carefully regular" }
 { "l_orderkey": 708, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37553.04, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-04", "l_receiptdate": "1998-08-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ests. even, regular hockey p" }
+{ "l_orderkey": 802, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41725.6, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y bold accou" }
 { "l_orderkey": 807, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31294.2, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "cial accoun" }
 { "l_orderkey": 871, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 27121.64, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-01-05", "l_receiptdate": "1996-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "symptotes use quickly near the " }
 { "l_orderkey": 1154, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 32337.34, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithely. final, blithe " }
+{ "l_orderkey": 1285, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 46941.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-05", "l_commitdate": "1992-08-08", "l_receiptdate": "1992-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special requests haggle blithely." }
 { "l_orderkey": 1347, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35466.76, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-25", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "r packages. f" }
+{ "l_orderkey": 1542, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21905.94, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-13", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-11-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending foxes nag blithely " }
+{ "l_orderkey": 1638, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 26078.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-06", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gle final, ironic pinto beans. " }
+{ "l_orderkey": 1763, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3129.42, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ut the slyly pending deposi" }
+{ "l_orderkey": 2176, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2086.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s pinto beans" }
 { "l_orderkey": 2180, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23992.22, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ggle alongside of the fluffily speci" }
 { "l_orderkey": 2307, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7301.98, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-09-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages cajo" }
+{ "l_orderkey": 2498, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 50070.72, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-25", "l_commitdate": "1994-01-09", "l_receiptdate": "1993-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "onic requests wake" }
+{ "l_orderkey": 2913, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle. even, bold instructi" }
 { "l_orderkey": 3109, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 52157.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-16", "l_receiptdate": "1993-10-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular packages boost blithely even, re" }
+{ "l_orderkey": 3458, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37553.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s lose. blithely ironic requests boost" }
+{ "l_orderkey": 3524, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17733.38, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "g, final epitaphs about the pinto " }
+{ "l_orderkey": 3553, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4172.56, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-13", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "olites boost bli" }
+{ "l_orderkey": 4198, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47984.44, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits among th" }
+{ "l_orderkey": 4834, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39639.32, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-06", "l_receiptdate": "1997-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "alongside of the carefully even plate" }
+{ "l_orderkey": 4897, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35466.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts. special dependencies use fluffily " }
 { "l_orderkey": 4964, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18776.52, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " platelets. furio" }
 { "l_orderkey": 5027, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3129.42, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-30", "l_commitdate": "1997-11-26", "l_receiptdate": "1997-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t the even mu" }
-{ "l_orderkey": 263, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 52157.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "re the packages. special" }
-{ "l_orderkey": 802, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41725.6, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y bold accou" }
-{ "l_orderkey": 2176, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2086.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-26", "l_commitdate": "1993-01-08", "l_receiptdate": "1993-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s pinto beans" }
-{ "l_orderkey": 3524, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17733.38, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "g, final epitaphs about the pinto " }
-{ "l_orderkey": 4897, "l_partkey": 143, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35466.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts. special dependencies use fluffily " }
 { "l_orderkey": 5094, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19819.66, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-06-12", "l_receiptdate": "1993-04-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ronic foxes. furi" }
+{ "l_orderkey": 5348, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14603.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "en pinto beans. somas cajo" }
 { "l_orderkey": 5511, "l_partkey": 143, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23992.22, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-03", "l_commitdate": "1995-01-05", "l_receiptdate": "1995-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ully deposits. warthogs hagg" }
 { "l_orderkey": 5543, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14603.96, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-09", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-10-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial reque" }
+{ "l_orderkey": 5729, "l_partkey": 143, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5215.7, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1994-11-11", "l_receiptdate": "1994-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. even sheaves nag courts. " }
+{ "l_orderkey": 5921, "l_partkey": 143, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 42768.74, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual, regular theodol" }
+{ "l_orderkey": 192, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "equests. ideas sleep idea" }
 { "l_orderkey": 225, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 45854.16, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-22", "l_commitdate": "1995-08-16", "l_receiptdate": "1995-10-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "leep slyly " }
 { "l_orderkey": 390, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 43769.88, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "counts nag across the sly, sil" }
-{ "l_orderkey": 1605, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-29", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". carefully r" }
-{ "l_orderkey": 2081, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-19", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " silent, spe" }
-{ "l_orderkey": 2241, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9379.26, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lyly final " }
-{ "l_orderkey": 3136, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31264.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "leep blithel" }
-{ "l_orderkey": 3556, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-14", "l_commitdate": "1992-12-21", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ckages boost quickl" }
-{ "l_orderkey": 3719, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-15", "l_receiptdate": "1997-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the furiously special pinto bean" }
-{ "l_orderkey": 4231, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "hely along the silent at" }
-{ "l_orderkey": 5318, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "requests must sleep slyly quickly" }
-{ "l_orderkey": 5413, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38559.18, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-01", "l_receiptdate": "1997-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly bold instructions affix idly unusual, " }
-{ "l_orderkey": 5670, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11463.54, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "etect furiously among the even pin" }
-{ "l_orderkey": 192, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1998-01-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "equests. ideas sleep idea" }
-{ "l_orderkey": 2306, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 43769.88, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "furiously final acco" }
-{ "l_orderkey": 2816, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4168.56, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-07", "l_receiptdate": "1995-01-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely pending id" }
-{ "l_orderkey": 3107, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36474.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-19", "l_receiptdate": "1997-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ets doubt furiously final ideas. final" }
-{ "l_orderkey": 1632, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. blithe, bold ideas cajo" }
-{ "l_orderkey": 1952, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-05-21", "l_receiptdate": "1994-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "packages haggle. " }
-{ "l_orderkey": 3395, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21884.94, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-01-13", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " careful dep" }
-{ "l_orderkey": 4036, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "equests wake about the bold id" }
 { "l_orderkey": 583, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34390.62, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-04-25", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "kages cajole slyly across the" }
 { "l_orderkey": 998, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31264.2, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-02", "l_commitdate": "1995-01-23", "l_receiptdate": "1994-12-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lyly idle Tir" }
 { "l_orderkey": 1153, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "oss the ex" }
+{ "l_orderkey": 1605, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-29", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-05-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". carefully r" }
+{ "l_orderkey": 1632, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. blithe, bold ideas cajo" }
+{ "l_orderkey": 1952, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-05-21", "l_receiptdate": "1994-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "packages haggle. " }
+{ "l_orderkey": 2081, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-19", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " silent, spe" }
+{ "l_orderkey": 2241, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9379.26, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-29", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lyly final " }
+{ "l_orderkey": 2306, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 43769.88, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-08-25", "l_receiptdate": "1995-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "furiously final acco" }
 { "l_orderkey": 2307, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 25011.36, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "stealthily special packages nag a" }
+{ "l_orderkey": 2816, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4168.56, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-07", "l_receiptdate": "1995-01-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". blithely pending id" }
+{ "l_orderkey": 3107, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36474.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-19", "l_receiptdate": "1997-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ets doubt furiously final ideas. final" }
+{ "l_orderkey": 3136, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31264.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "leep blithel" }
+{ "l_orderkey": 3395, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21884.94, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-19", "l_commitdate": "1995-01-13", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " careful dep" }
 { "l_orderkey": 3458, "l_partkey": 142, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-02-02", "l_receiptdate": "1995-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites; regular theodolites cajole " }
+{ "l_orderkey": 3556, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46896.3, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-14", "l_commitdate": "1992-12-21", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ckages boost quickl" }
+{ "l_orderkey": 3719, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44812.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-15", "l_receiptdate": "1997-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "the furiously special pinto bean" }
+{ "l_orderkey": 4036, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6252.84, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "equests wake about the bold id" }
+{ "l_orderkey": 4231, "l_partkey": 142, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48980.58, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-27", "l_commitdate": "1998-01-26", "l_receiptdate": "1997-12-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "hely along the silent at" }
 { "l_orderkey": 5158, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42727.74, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-19", "l_receiptdate": "1997-03-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "deposits. quickly special " }
-{ "l_orderkey": 1344, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15617.1, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rding to the blithely ironic theodolite" }
-{ "l_orderkey": 2373, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30193.06, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-14", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent ideas affix furiousl" }
-{ "l_orderkey": 2820, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39563.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests despite the carefully unusual a" }
-{ "l_orderkey": 3367, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35398.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-02-23", "l_receiptdate": "1993-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts wake slyly " }
-{ "l_orderkey": 3712, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28110.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ctions. even accounts haggle alongside " }
-{ "l_orderkey": 3751, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "rthogs could have to slee" }
-{ "l_orderkey": 4067, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1992-11-23", "l_receiptdate": "1993-01-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle slyly unusual, final" }
-{ "l_orderkey": 4228, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "f the slyly fluffy pinto beans are" }
-{ "l_orderkey": 4448, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3123.42, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ronic theod" }
-{ "l_orderkey": 4485, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47892.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1994-12-14", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". ironic foxes haggle. regular war" }
-{ "l_orderkey": 5252, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13534.82, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "boost fluffily across " }
-{ "l_orderkey": 5536, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11452.54, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-18", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " snooze furio" }
+{ "l_orderkey": 5318, "l_partkey": 142, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32306.34, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "requests must sleep slyly quickly" }
+{ "l_orderkey": 5413, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38559.18, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-01-01", "l_receiptdate": "1997-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly bold instructions affix idly unusual, " }
+{ "l_orderkey": 5670, "l_partkey": 142, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11463.54, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "etect furiously among the even pin" }
+{ "l_orderkey": 132, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. platelets wake furio" }
 { "l_orderkey": 321, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 42686.74, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "special packages shall have to doze blit" }
 { "l_orderkey": 582, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43727.88, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-12-21", "l_receiptdate": "1997-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "iously beside the silent de" }
+{ "l_orderkey": 738, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-06-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ove the slyly regular p" }
 { "l_orderkey": 836, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47892.44, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-04-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "boldly final pinto beans haggle furiously" }
+{ "l_orderkey": 1089, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1041.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-07-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "n courts among the caref" }
+{ "l_orderkey": 1344, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15617.1, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rding to the blithely ironic theodolite" }
+{ "l_orderkey": 1380, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41645.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly final frets. ironic," }
 { "l_orderkey": 1604, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38522.18, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-09-21", "l_receiptdate": "1993-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "requests. blithely ironic somas s" }
-{ "l_orderkey": 2086, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-15", "l_commitdate": "1995-01-05", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e carefully along th" }
-{ "l_orderkey": 3264, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44769.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep at the blithely bold" }
-{ "l_orderkey": 3747, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43727.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y. blithely fina" }
-{ "l_orderkey": 3776, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 51015.86, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-16", "l_receiptdate": "1992-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "equests. final, thin grouches " }
-{ "l_orderkey": 4672, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-25", "l_receiptdate": "1995-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s boost at the ca" }
-{ "l_orderkey": 5409, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-13", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "cross the sil" }
 { "l_orderkey": 1664, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 41645.6, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-02", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se blithely unusual pains. carefully" }
 { "l_orderkey": 1730, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44769.02, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-26", "l_commitdate": "1998-10-22", "l_receiptdate": "1998-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ng deposits cajo" }
-{ "l_orderkey": 2311, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " fluffily even patterns haggle blithely. re" }
-{ "l_orderkey": 3876, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y above the pending tithes. blithely ironi" }
-{ "l_orderkey": 4449, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10411.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-09", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-05-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts alongside of the platelets integr" }
-{ "l_orderkey": 4583, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "romise. reques" }
-{ "l_orderkey": 132, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. platelets wake furio" }
-{ "l_orderkey": 738, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-06-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ove the slyly regular p" }
-{ "l_orderkey": 1089, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1041.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-07-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "n courts among the caref" }
-{ "l_orderkey": 1380, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41645.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly final frets. ironic," }
 { "l_orderkey": 1890, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27069.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ngage. slyly ironic " }
+{ "l_orderkey": 2086, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-15", "l_commitdate": "1995-01-05", "l_receiptdate": "1994-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e carefully along th" }
+{ "l_orderkey": 2311, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18740.52, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-07-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " fluffily even patterns haggle blithely. re" }
+{ "l_orderkey": 2373, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30193.06, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-14", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent ideas affix furiousl" }
 { "l_orderkey": 2496, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39563.32, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " bold accounts. furi" }
 { "l_orderkey": 2565, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 49974.72, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-05-06", "l_receiptdate": "1998-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "r instructions sleep qui" }
+{ "l_orderkey": 2820, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39563.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests despite the carefully unusual a" }
+{ "l_orderkey": 3264, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 44769.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep at the blithely bold" }
+{ "l_orderkey": 3367, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35398.76, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-30", "l_commitdate": "1993-02-23", "l_receiptdate": "1993-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts wake slyly " }
+{ "l_orderkey": 3712, "l_partkey": 141, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 28110.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ctions. even accounts haggle alongside " }
+{ "l_orderkey": 3747, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 43727.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-11-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y. blithely fina" }
+{ "l_orderkey": 3751, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "rthogs could have to slee" }
+{ "l_orderkey": 3776, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 51015.86, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1993-02-16", "l_receiptdate": "1992-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "equests. final, thin grouches " }
+{ "l_orderkey": 3876, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12493.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y above the pending tithes. blithely ironi" }
+{ "l_orderkey": 4067, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-26", "l_commitdate": "1992-11-23", "l_receiptdate": "1993-01-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ts haggle slyly unusual, final" }
 { "l_orderkey": 4166, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8329.12, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "uickly. blithely pending de" }
+{ "l_orderkey": 4228, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "f the slyly fluffy pinto beans are" }
+{ "l_orderkey": 4448, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3123.42, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ronic theod" }
+{ "l_orderkey": 4449, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10411.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-09", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-05-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ccounts alongside of the platelets integr" }
+{ "l_orderkey": 4485, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47892.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1994-12-14", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". ironic foxes haggle. regular war" }
 { "l_orderkey": 4512, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 33316.48, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "counts are against the quickly regular " }
+{ "l_orderkey": 4583, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "romise. reques" }
+{ "l_orderkey": 4672, "l_partkey": 141, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20822.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-25", "l_receiptdate": "1995-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s boost at the ca" }
 { "l_orderkey": 4901, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38522.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-26", "l_commitdate": "1998-02-20", "l_receiptdate": "1998-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " furiously ev" }
-{ "l_orderkey": 1058, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24963.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully ironic accounts. express accou" }
-{ "l_orderkey": 1155, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12481.68, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1998-01-03", "l_receiptdate": "1997-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages do" }
-{ "l_orderkey": 2116, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic dependencies around the iro" }
-{ "l_orderkey": 2307, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously. furiously furious requ" }
-{ "l_orderkey": 2881, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7280.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ironic packages are carefully final ac" }
-{ "l_orderkey": 3618, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39525.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-22", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nts haggle fluffily above the regular " }
-{ "l_orderkey": 4322, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9361.26, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ual instructio" }
-{ "l_orderkey": 4386, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4160.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully iron" }
-{ "l_orderkey": 4896, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-12-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e after the slowly f" }
+{ "l_orderkey": 5252, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13534.82, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-02", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "boost fluffily across " }
+{ "l_orderkey": 5409, "l_partkey": 141, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17699.38, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-13", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "cross the sil" }
+{ "l_orderkey": 5536, "l_partkey": 141, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11452.54, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-18", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " snooze furio" }
 { "l_orderkey": 6, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "p furiously special foxes" }
 { "l_orderkey": 68, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42645.74, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-07-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "eposits nag special ideas. furiousl" }
-{ "l_orderkey": 2690, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of th" }
-{ "l_orderkey": 3110, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "side of the blithely unusual courts. slyly " }
-{ "l_orderkey": 4320, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6240.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-01-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "against the carefully careful asym" }
-{ "l_orderkey": 4871, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 10401.4, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "p ironic theodolites. slyly even platel" }
-{ "l_orderkey": 4934, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8321.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-04-30", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "arefully express pains cajo" }
-{ "l_orderkey": 5506, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-01-13", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "onic theodolites are fluffil" }
-{ "l_orderkey": 2757, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26003.5, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-11-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uickly regular " }
-{ "l_orderkey": 3296, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-12-20", "l_receiptdate": "1994-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular deposits. quic" }
-{ "l_orderkey": 4102, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-06", "l_receiptdate": "1996-06-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y among the furiously special" }
-{ "l_orderkey": 5092, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13521.82, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1996-01-05", "l_receiptdate": "1995-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es detect sly" }
-{ "l_orderkey": 5698, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. even, ironic " }
-{ "l_orderkey": 5955, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14561.96, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " unusual, bold theodolit" }
 { "l_orderkey": 165, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 50966.86, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-20", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "uses sleep slyly ruthlessly regular a" }
+{ "l_orderkey": 1058, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24963.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully ironic accounts. express accou" }
+{ "l_orderkey": 1155, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12481.68, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1998-01-03", "l_receiptdate": "1997-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages do" }
 { "l_orderkey": 1218, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16642.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-08-07", "l_receiptdate": "1994-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ven realms be" }
 { "l_orderkey": 1537, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3120.42, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-20", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s, final ideas detect sl" }
 { "l_orderkey": 1700, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39525.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ular dependencies engage slyly " }
 { "l_orderkey": 1921, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21842.94, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-03-28", "l_receiptdate": "1994-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ckly regula" }
+{ "l_orderkey": 2116, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic dependencies around the iro" }
 { "l_orderkey": 2211, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 41605.6, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-10-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "posits among the express dolphins" }
+{ "l_orderkey": 2307, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-08-22", "l_receiptdate": "1993-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ously. furiously furious requ" }
+{ "l_orderkey": 2690, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly alongside of th" }
+{ "l_orderkey": 2757, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 26003.5, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-11-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uickly regular " }
+{ "l_orderkey": 2881, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7280.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ironic packages are carefully final ac" }
+{ "l_orderkey": 3110, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-09", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "side of the blithely unusual courts. slyly " }
+{ "l_orderkey": 3296, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48886.58, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-12-20", "l_receiptdate": "1994-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular deposits. quic" }
 { "l_orderkey": 3335, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16642.24, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "g packages. carefully regular reque" }
+{ "l_orderkey": 3618, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39525.32, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-22", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nts haggle fluffily above the regular " }
 { "l_orderkey": 3623, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13521.82, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-02-26", "l_receiptdate": "1997-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "deas. furiously expres" }
 { "l_orderkey": 3872, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 41605.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1996-10-29", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nts? regularly ironic ex" }
 { "l_orderkey": 3873, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30164.06, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-22", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "olphins af" }
 { "l_orderkey": 3876, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-30", "l_commitdate": "1996-10-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "t dependencies. blithely final packages u" }
+{ "l_orderkey": 4102, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 40565.46, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-06", "l_receiptdate": "1996-06-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y among the furiously special" }
+{ "l_orderkey": 4320, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6240.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-01-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "against the carefully careful asym" }
+{ "l_orderkey": 4322, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9361.26, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ual instructio" }
+{ "l_orderkey": 4386, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4160.56, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully iron" }
+{ "l_orderkey": 4871, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 10401.4, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "p ironic theodolites. slyly even platel" }
+{ "l_orderkey": 4896, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45766.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-12-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e after the slowly f" }
+{ "l_orderkey": 4934, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8321.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-04-30", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "arefully express pains cajo" }
+{ "l_orderkey": 5092, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13521.82, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1996-01-05", "l_receiptdate": "1995-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es detect sly" }
+{ "l_orderkey": 5506, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2080.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-01-13", "l_receiptdate": "1994-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "onic theodolites are fluffil" }
 { "l_orderkey": 5669, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 31204.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-07-28", "l_receiptdate": "1996-08-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l accounts. care" }
-{ "l_orderkey": 1696, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13508.69, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-01", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-03-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tructions play slyly q" }
-{ "l_orderkey": 2179, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20782.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ncies. fin" }
-{ "l_orderkey": 2823, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 49878.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously busily slow excus" }
-{ "l_orderkey": 3328, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45721.72, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-12-19", "l_receiptdate": "1992-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "dly quickly final foxes? re" }
-{ "l_orderkey": 4992, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23899.99, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-28", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly regul" }
-{ "l_orderkey": 5252, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gular requests." }
-{ "l_orderkey": 1059, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38447.81, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " packages lose in place of the slyly unusu" }
-{ "l_orderkey": 1735, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50917.37, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1993-02-03", "l_receiptdate": "1993-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y express accounts above the exp" }
-{ "l_orderkey": 2177, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28056.51, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "even, regula" }
-{ "l_orderkey": 3074, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-08", "l_commitdate": "1993-01-28", "l_receiptdate": "1992-12-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously throu" }
-{ "l_orderkey": 3747, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31173.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-15", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "! furiously f" }
-{ "l_orderkey": 4099, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "onic foxes. quickly final fox" }
-{ "l_orderkey": 4647, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2078.26, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-07-22", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dolites wake furiously special pinto be" }
-{ "l_orderkey": 5732, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27017.38, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "totes cajole according to the theodolites." }
-{ "l_orderkey": 5765, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32213.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the furiou" }
+{ "l_orderkey": 5698, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38485.18, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. even, ironic " }
+{ "l_orderkey": 5955, "l_partkey": 140, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14561.96, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " unusual, bold theodolit" }
 { "l_orderkey": 101, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12469.56, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-29", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-04-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". quickly regular" }
 { "l_orderkey": 544, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48839.11, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-03-27", "l_receiptdate": "1993-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial pains. deposits grow foxes. " }
 { "l_orderkey": 710, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "xpress, special ideas. bl" }
 { "l_orderkey": 867, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1993-12-25", "l_receiptdate": "1994-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "pendencies-- slyly unusual packages hagg" }
-{ "l_orderkey": 2309, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47799.98, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly according to the carefully " }
-{ "l_orderkey": 2853, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14547.82, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oach slyly along t" }
-{ "l_orderkey": 4578, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "odolites. carefully unusual ideas accor" }
+{ "l_orderkey": 1059, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 38447.81, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " packages lose in place of the slyly unusu" }
+{ "l_orderkey": 1696, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13508.69, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-01", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-03-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "tructions play slyly q" }
 { "l_orderkey": 1731, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-11", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fily quick asymptotes" }
+{ "l_orderkey": 1735, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50917.37, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-31", "l_commitdate": "1993-02-03", "l_receiptdate": "1993-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y express accounts above the exp" }
 { "l_orderkey": 1956, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-11-26", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r theodolites sleep above the b" }
+{ "l_orderkey": 2177, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28056.51, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "even, regula" }
+{ "l_orderkey": 2179, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20782.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ncies. fin" }
+{ "l_orderkey": 2309, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47799.98, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly according to the carefully " }
 { "l_orderkey": 2534, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 30134.77, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-09", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ugouts haggle slyly. final" }
 { "l_orderkey": 2596, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44682.59, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ial packages haggl" }
+{ "l_orderkey": 2823, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 49878.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-21", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously busily slow excus" }
+{ "l_orderkey": 2853, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14547.82, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oach slyly along t" }
 { "l_orderkey": 2880, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 27017.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-15", "l_receiptdate": "1992-04-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ully among the regular warthogs" }
 { "l_orderkey": 3013, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31173.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ronic packages. slyly even" }
+{ "l_orderkey": 3074, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-08", "l_commitdate": "1993-01-28", "l_receiptdate": "1992-12-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously throu" }
 { "l_orderkey": 3171, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51956.5, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously final foxes about the ca" }
+{ "l_orderkey": 3328, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45721.72, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-12-19", "l_receiptdate": "1992-12-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "dly quickly final foxes? re" }
 { "l_orderkey": 3427, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 41565.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-08-19", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "patterns cajole ca" }
+{ "l_orderkey": 3747, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31173.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-15", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "! furiously f" }
 { "l_orderkey": 3910, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10391.3, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-10-31", "l_receiptdate": "1996-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tions boost furiously unusual e" }
 { "l_orderkey": 4066, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9352.17, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "nal, ironic accounts. blithel" }
+{ "l_orderkey": 4099, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-09-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "onic foxes. quickly final fox" }
 { "l_orderkey": 4354, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18704.34, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-12-11", "l_receiptdate": "1994-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ross the furiously " }
+{ "l_orderkey": 4578, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7273.91, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "odolites. carefully unusual ideas accor" }
+{ "l_orderkey": 4647, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2078.26, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-07-22", "l_receiptdate": "1994-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "dolites wake furiously special pinto be" }
+{ "l_orderkey": 4992, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23899.99, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-28", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly regul" }
 { "l_orderkey": 5251, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37408.68, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "slowly! bli" }
+{ "l_orderkey": 5252, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40526.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gular requests." }
+{ "l_orderkey": 5732, "l_partkey": 139, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 27017.38, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-18", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "totes cajole according to the theodolites." }
+{ "l_orderkey": 5765, "l_partkey": 139, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32213.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "the furiou" }
 { "l_orderkey": 33, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5190.65, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". stealthily bold exc" }
-{ "l_orderkey": 448, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-10-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ious, final gifts" }
-{ "l_orderkey": 2340, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-01", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": ". carefully ironic" }
-{ "l_orderkey": 2848, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8305.04, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-07-09", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly regular foxes. " }
-{ "l_orderkey": 3329, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-06", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts at the re" }
-{ "l_orderkey": 3396, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly special foxes. accounts wake careful" }
-{ "l_orderkey": 4832, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44639.59, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-02-20", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oze according to the accou" }
 { "l_orderkey": 69, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17648.21, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-02", "l_commitdate": "1994-07-07", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "final, pending instr" }
+{ "l_orderkey": 226, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47753.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. carefully bold accounts cajol" }
 { "l_orderkey": 291, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19724.47, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-04-25", "l_receiptdate": "1994-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e. ruthlessly final accounts after the" }
+{ "l_orderkey": 448, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-10-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ious, final gifts" }
+{ "l_orderkey": 482, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33220.16, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "usual deposits affix against " }
+{ "l_orderkey": 576, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5190.65, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l foxes boost slyly. accounts af" }
+{ "l_orderkey": 1125, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1994-12-02", "l_receiptdate": "1995-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es about the slyly s" }
+{ "l_orderkey": 1281, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "dencies. thinly final pinto beans wake" }
+{ "l_orderkey": 1283, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests use along the fluff" }
+{ "l_orderkey": 1889, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l pinto beans kindle " }
 { "l_orderkey": 1991, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6228.78, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uickly blithely final de" }
 { "l_orderkey": 2272, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 31143.9, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-08-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "quests at the foxes haggle evenly pack" }
+{ "l_orderkey": 2309, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 49830.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts. id" }
+{ "l_orderkey": 2340, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-01", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": ". carefully ironic" }
+{ "l_orderkey": 2848, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8305.04, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-07-09", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly regular foxes. " }
+{ "l_orderkey": 2951, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ep about the final, even package" }
+{ "l_orderkey": 3010, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ounts. pendin" }
+{ "l_orderkey": 3235, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffy pinto bea" }
+{ "l_orderkey": 3329, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-06", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts at the re" }
 { "l_orderkey": 3362, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-08-28", "l_receiptdate": "1995-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "es against the quickly permanent pint" }
+{ "l_orderkey": 3396, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9343.17, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly special foxes. accounts wake careful" }
+{ "l_orderkey": 3525, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28029.51, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y slyly special asymptotes" }
+{ "l_orderkey": 3616, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. furiously ev" }
 { "l_orderkey": 4007, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15571.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "le furiously quickly " }
+{ "l_orderkey": 4065, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14533.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-22", "l_commitdate": "1994-07-29", "l_receiptdate": "1994-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e furiously outside " }
+{ "l_orderkey": 4132, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pths wake against the stealthily special pi" }
+{ "l_orderkey": 4161, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43601.46, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-12", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thely across the even attainments. express" }
+{ "l_orderkey": 4487, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38410.81, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "bove the fu" }
+{ "l_orderkey": 4513, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35296.42, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sits. quickly even instructions " }
+{ "l_orderkey": 4832, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44639.59, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-02-20", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "oze according to the accou" }
+{ "l_orderkey": 4934, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aggle furiously among the busily final re" }
+{ "l_orderkey": 4965, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "iously slyly" }
+{ "l_orderkey": 5157, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-06", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-09-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits nag blithely. final reque" }
+{ "l_orderkey": 5189, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45677.72, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-02-07", "l_receiptdate": "1994-01-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y finally pendin" }
 { "l_orderkey": 5248, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 46715.85, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-09", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". bold, pending foxes h" }
+{ "l_orderkey": 5286, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-12-21", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. express foxes of the" }
+{ "l_orderkey": 5479, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51906.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-24", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic gifts. even dependencies sno" }
 { "l_orderkey": 5507, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49830.24, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly idle deposits. final, final fox" }
 { "l_orderkey": 5573, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44639.59, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " bold package" }
 { "l_orderkey": 5638, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46715.85, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-17", "l_commitdate": "1994-03-09", "l_receiptdate": "1994-06-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar foxes. fluffily pending accounts " }
 { "l_orderkey": 5664, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-29", "l_commitdate": "1998-09-17", "l_receiptdate": "1998-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "d the final " }
-{ "l_orderkey": 482, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33220.16, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "usual deposits affix against " }
-{ "l_orderkey": 1125, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1994-12-02", "l_receiptdate": "1995-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es about the slyly s" }
-{ "l_orderkey": 1283, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-11-07", "l_receiptdate": "1996-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests use along the fluff" }
-{ "l_orderkey": 2309, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 49830.24, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts. id" }
-{ "l_orderkey": 2951, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ep about the final, even package" }
-{ "l_orderkey": 4132, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pths wake against the stealthily special pi" }
-{ "l_orderkey": 4487, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38410.81, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "bove the fu" }
-{ "l_orderkey": 4513, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35296.42, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sits. quickly even instructions " }
-{ "l_orderkey": 4934, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aggle furiously among the busily final re" }
-{ "l_orderkey": 4965, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "iously slyly" }
-{ "l_orderkey": 5157, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18686.34, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-06", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-09-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits nag blithely. final reque" }
-{ "l_orderkey": 5479, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51906.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-24", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic gifts. even dependencies sno" }
-{ "l_orderkey": 226, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47753.98, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. carefully bold accounts cajol" }
-{ "l_orderkey": 576, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5190.65, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l foxes boost slyly. accounts af" }
-{ "l_orderkey": 1281, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34258.29, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "dencies. thinly final pinto beans wake" }
-{ "l_orderkey": 1889, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37372.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l pinto beans kindle " }
-{ "l_orderkey": 3010, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23876.99, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-03-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ounts. pendin" }
-{ "l_orderkey": 3235, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 30105.77, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-28", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffy pinto bea" }
-{ "l_orderkey": 3525, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 28029.51, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y slyly special asymptotes" }
-{ "l_orderkey": 3616, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29067.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-20", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. furiously ev" }
-{ "l_orderkey": 4065, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14533.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-22", "l_commitdate": "1994-07-29", "l_receiptdate": "1994-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e furiously outside " }
-{ "l_orderkey": 4161, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43601.46, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-12", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thely across the even attainments. express" }
-{ "l_orderkey": 5189, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45677.72, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-02-07", "l_receiptdate": "1994-01-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y finally pendin" }
-{ "l_orderkey": 5286, "l_partkey": 138, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24915.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-27", "l_commitdate": "1997-12-21", "l_receiptdate": "1997-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. express foxes of the" }
 { "l_orderkey": 135, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20742.6, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-02-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "theodolites. quickly p" }
-{ "l_orderkey": 1760, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "instructions poach slyly ironic theodolites" }
-{ "l_orderkey": 4865, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4148.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-24", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sts. blithely special instruction" }
-{ "l_orderkey": 1095, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34225.29, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-09-22", "l_receiptdate": "1995-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly around the iron" }
-{ "l_orderkey": 1703, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he carefully" }
-{ "l_orderkey": 1795, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-28", "l_commitdate": "1994-05-24", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ites sleep carefully slyly p" }
-{ "l_orderkey": 2277, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully bold" }
-{ "l_orderkey": 3111, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22816.86, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. regular dolphins against the " }
-{ "l_orderkey": 3940, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11408.43, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e of the special packages. furiously" }
-{ "l_orderkey": 4290, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23853.99, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uests cajole carefully." }
-{ "l_orderkey": 5696, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29039.64, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-03", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the fluffily brave pearls " }
-{ "l_orderkey": 5794, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48745.11, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "quests. blithely final excu" }
 { "l_orderkey": 387, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1037.13, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-05-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " pinto beans wake furiously carefu" }
 { "l_orderkey": 485, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 22816.86, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "refully final notornis haggle according " }
 { "l_orderkey": 547, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "thely specia" }
-{ "l_orderkey": 1029, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46670.85, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sits boost blithely" }
-{ "l_orderkey": 1062, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-03-07", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. pending acc" }
-{ "l_orderkey": 1573, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7259.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-30", "l_commitdate": "1993-03-14", "l_receiptdate": "1993-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eodolites sleep slyly. slyly f" }
-{ "l_orderkey": 1955, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g to the carefully sile" }
-{ "l_orderkey": 2309, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9334.17, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-10", "l_receiptdate": "1996-01-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ding, unusual instructions. dep" }
-{ "l_orderkey": 2656, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions wake along the furio" }
-{ "l_orderkey": 3429, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " haggle furiously ir" }
-{ "l_orderkey": 3524, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5185.65, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts whithout the bold depende" }
-{ "l_orderkey": 3936, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25928.25, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-03", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gular requests nag quic" }
-{ "l_orderkey": 4099, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3111.39, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". special packages sleep" }
-{ "l_orderkey": 4484, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40448.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-04-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic accounts wake blithel" }
-{ "l_orderkey": 4612, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 41485.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special platelets." }
-{ "l_orderkey": 5635, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly even" }
 { "l_orderkey": 614, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-01-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " regular platelets cajole quickly eve" }
 { "l_orderkey": 675, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-19", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. furiously expre" }
 { "l_orderkey": 993, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-11-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es. ironic, ironic requests" }
+{ "l_orderkey": 1029, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46670.85, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sits boost blithely" }
+{ "l_orderkey": 1062, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-03-07", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. pending acc" }
+{ "l_orderkey": 1095, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 34225.29, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-09-22", "l_receiptdate": "1995-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly around the iron" }
+{ "l_orderkey": 1573, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7259.91, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-30", "l_commitdate": "1993-03-14", "l_receiptdate": "1993-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eodolites sleep slyly. slyly f" }
+{ "l_orderkey": 1703, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36299.55, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he carefully" }
+{ "l_orderkey": 1760, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "instructions poach slyly ironic theodolites" }
+{ "l_orderkey": 1795, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45633.72, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-28", "l_commitdate": "1994-05-24", "l_receiptdate": "1994-05-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ites sleep carefully slyly p" }
+{ "l_orderkey": 1955, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g to the carefully sile" }
+{ "l_orderkey": 2277, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully bold" }
+{ "l_orderkey": 2309, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9334.17, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-10", "l_receiptdate": "1996-01-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ding, unusual instructions. dep" }
+{ "l_orderkey": 2656, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 39410.94, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions wake along the furio" }
 { "l_orderkey": 2661, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42522.33, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-03-27", "l_receiptdate": "1997-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "iously ironically ironic requests. " }
 { "l_orderkey": 2753, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 37336.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle slyly final c" }
+{ "l_orderkey": 3111, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22816.86, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests. regular dolphins against the " }
+{ "l_orderkey": 3429, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49782.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " haggle furiously ir" }
+{ "l_orderkey": 3524, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5185.65, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts whithout the bold depende" }
 { "l_orderkey": 3617, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 11408.43, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly quickly even requests. final" }
 { "l_orderkey": 3652, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38373.81, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "osits haggle carefu" }
 { "l_orderkey": 3745, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18668.34, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-16", "l_receiptdate": "1993-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slyly bold pinto beans according to " }
 { "l_orderkey": 3810, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42522.33, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-26", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l requests boost slyly along the slyl" }
+{ "l_orderkey": 3936, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25928.25, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-03", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gular requests nag quic" }
+{ "l_orderkey": 3940, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11408.43, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-09", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e of the special packages. furiously" }
+{ "l_orderkey": 4099, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3111.39, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-10-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". special packages sleep" }
 { "l_orderkey": 4102, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7259.91, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "bove the carefully pending the" }
+{ "l_orderkey": 4290, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23853.99, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uests cajole carefully." }
+{ "l_orderkey": 4484, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40448.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-04-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic accounts wake blithel" }
+{ "l_orderkey": 4612, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 41485.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special platelets." }
+{ "l_orderkey": 4865, "l_partkey": 137, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4148.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-24", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "sts. blithely special instruction" }
+{ "l_orderkey": 5635, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 33188.16, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly even" }
+{ "l_orderkey": 5696, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 29039.64, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-03", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " the fluffily brave pearls " }
+{ "l_orderkey": 5794, "l_partkey": 137, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48745.11, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "quests. blithely final excu" }
+{ "l_orderkey": 96, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-03", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e quickly even ideas. furiou" }
 { "l_orderkey": 129, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 35228.42, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-29", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. express ideas" }
 { "l_orderkey": 196, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19686.47, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-04-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sts maintain foxes. furiously regular p" }
-{ "l_orderkey": 1574, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14505.82, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-19", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily bold a" }
-{ "l_orderkey": 1733, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39372.94, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-08-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " deposits " }
-{ "l_orderkey": 2084, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38336.81, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y careful courts." }
-{ "l_orderkey": 2786, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-19", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "low deposits are ironic" }
-{ "l_orderkey": 2915, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al requests haggle furiousl" }
-{ "l_orderkey": 2951, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24867.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-04-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " ironic multipliers. express, regular" }
-{ "l_orderkey": 3366, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ackages sleep carefully across the bli" }
-{ "l_orderkey": 3428, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 48698.11, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-05-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y final pinto " }
-{ "l_orderkey": 3650, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-26", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckly special platelets. furiously sil" }
-{ "l_orderkey": 4035, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4144.52, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en instructions sleep blith" }
-{ "l_orderkey": 4805, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18650.34, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "o use pending, unusu" }
-{ "l_orderkey": 5153, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43517.46, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-19", "l_commitdate": "1995-11-23", "l_receiptdate": "1995-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ickly even deposi" }
-{ "l_orderkey": 5316, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32120.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. deposits cajole around t" }
-{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-05-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ove the regula" }
-{ "l_orderkey": 5986, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6216.78, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al foxes within the slyly speci" }
-{ "l_orderkey": 1061, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42481.33, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-29", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s are. ironic theodolites cajole. dep" }
-{ "l_orderkey": 1831, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-01-27", "l_receiptdate": "1993-12-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "mptotes. furiously regular dolphins al" }
-{ "l_orderkey": 2373, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3108.39, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dependencies wake ironical" }
-{ "l_orderkey": 3106, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lets. quietly regular courts " }
-{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45589.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "efully ironi" }
-{ "l_orderkey": 96, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-03", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e quickly even ideas. furiou" }
 { "l_orderkey": 486, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26939.38, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-05-25", "l_receiptdate": "1996-03-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "forges along the " }
+{ "l_orderkey": 866, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-01-14", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tegrate fluffily. carefully f" }
 { "l_orderkey": 1027, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22794.86, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "the furiously express ex" }
+{ "l_orderkey": 1061, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 42481.33, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-29", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s are. ironic theodolites cajole. dep" }
 { "l_orderkey": 1095, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24867.12, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "packages nod furiously above the carefully " }
 { "l_orderkey": 1153, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 26939.38, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-16", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-09-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kages haggle carefully. f" }
 { "l_orderkey": 1312, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 29011.64, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-09", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously final frays should use quick" }
-{ "l_orderkey": 4384, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "instructions sleep. blithely express pa" }
-{ "l_orderkey": 5763, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23830.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "re after the blithel" }
-{ "l_orderkey": 866, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-01-14", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tegrate fluffily. carefully f" }
+{ "l_orderkey": 1574, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14505.82, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-19", "l_receiptdate": "1997-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ily bold a" }
+{ "l_orderkey": 1733, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 39372.94, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-08-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " deposits " }
+{ "l_orderkey": 1831, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-01-27", "l_receiptdate": "1993-12-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "mptotes. furiously regular dolphins al" }
+{ "l_orderkey": 2084, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38336.81, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-16", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y careful courts." }
+{ "l_orderkey": 2373, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3108.39, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dependencies wake ironical" }
+{ "l_orderkey": 2786, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-19", "l_commitdate": "1992-05-08", "l_receiptdate": "1992-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "low deposits are ironic" }
+{ "l_orderkey": 2915, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15541.95, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al requests haggle furiousl" }
+{ "l_orderkey": 2951, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24867.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-04-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " ironic multipliers. express, regular" }
+{ "l_orderkey": 3106, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lets. quietly regular courts " }
 { "l_orderkey": 3332, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21758.73, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1995-01-08", "l_receiptdate": "1995-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " quick packages sle" }
+{ "l_orderkey": 3366, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9325.17, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ackages sleep carefully across the bli" }
+{ "l_orderkey": 3428, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 48698.11, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-05-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y final pinto " }
+{ "l_orderkey": 3650, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-26", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckly special platelets. furiously sil" }
 { "l_orderkey": 3782, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 31083.9, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1996-10-31", "l_receiptdate": "1997-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "slyly even pinto beans hag" }
+{ "l_orderkey": 4035, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4144.52, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en instructions sleep blith" }
+{ "l_orderkey": 4384, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5180.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "instructions sleep. blithely express pa" }
 { "l_orderkey": 4707, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " alongside of the slyly ironic instructio" }
+{ "l_orderkey": 4805, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 18650.34, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "o use pending, unusu" }
 { "l_orderkey": 4871, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 37300.68, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-29", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ely according" }
-{ "l_orderkey": 1508, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30018.77, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r instructions. carefully" }
-{ "l_orderkey": 1826, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ously? quickly pe" }
-{ "l_orderkey": 2052, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ts according t" }
-{ "l_orderkey": 3172, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28983.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-07-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "regular ideas. packages are furi" }
-{ "l_orderkey": 3844, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es haggle final acco" }
-{ "l_orderkey": 5219, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " blithely according to the stea" }
-{ "l_orderkey": 5254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10351.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-19", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " accounts. silent deposit" }
+{ "l_orderkey": 5153, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43517.46, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-19", "l_commitdate": "1995-11-23", "l_receiptdate": "1995-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ickly even deposi" }
+{ "l_orderkey": 5316, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 32120.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. deposits cajole around t" }
+{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45589.72, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "efully ironi" }
+{ "l_orderkey": 5604, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50770.37, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-07-07", "l_receiptdate": "1998-05-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ove the regula" }
+{ "l_orderkey": 5763, "l_partkey": 136, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23830.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "re after the blithel" }
+{ "l_orderkey": 5986, "l_partkey": 136, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6216.78, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "al foxes within the slyly speci" }
+{ "l_orderkey": 99, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43475.46, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-19", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages are fluffily furiously ir" }
 { "l_orderkey": 353, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12421.56, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g deposits cajole " }
+{ "l_orderkey": 935, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37264.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "leep about the exp" }
 { "l_orderkey": 1095, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28983.64, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " regular pac" }
+{ "l_orderkey": 1254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckages boost. furious warhorses cajole" }
 { "l_orderkey": 1445, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24843.12, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-05-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rate after the carefully reg" }
+{ "l_orderkey": 1508, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 30018.77, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r instructions. carefully" }
 { "l_orderkey": 1699, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17597.21, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "haggle blithely slyly" }
+{ "l_orderkey": 1826, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ously? quickly pe" }
 { "l_orderkey": 1925, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-11", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts. carefully ironic packages boost ab" }
+{ "l_orderkey": 2052, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ts according t" }
+{ "l_orderkey": 2055, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12421.56, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al pains. acco" }
 { "l_orderkey": 2211, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3105.39, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-28", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pendencies after the regular f" }
 { "l_orderkey": 2276, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13456.69, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-24", "l_commitdate": "1996-06-18", "l_receiptdate": "1996-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully ironic foxes cajole q" }
+{ "l_orderkey": 2532, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1035.13, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-11-23", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely final ideas cajole despite the ca" }
 { "l_orderkey": 2567, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 44510.59, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "requests. final courts cajole " }
-{ "l_orderkey": 4422, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5175.65, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e furiously about t" }
-{ "l_orderkey": 4486, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. specia" }
-{ "l_orderkey": 5959, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 50721.37, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usual packages haggle slyly pi" }
-{ "l_orderkey": 935, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37264.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "leep about the exp" }
-{ "l_orderkey": 2055, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12421.56, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-11-23", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al pains. acco" }
+{ "l_orderkey": 2659, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24843.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle carefully " }
+{ "l_orderkey": 3172, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28983.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-07-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "regular ideas. packages are furi" }
 { "l_orderkey": 3232, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35194.42, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-14", "l_receiptdate": "1993-02-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "old packages integrate quickly " }
 { "l_orderkey": 3684, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13456.69, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-23", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-08-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing, unusual pinto beans! thinly p" }
-{ "l_orderkey": 5572, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-08", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully regular platelet" }
-{ "l_orderkey": 99, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43475.46, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-19", "l_commitdate": "1994-05-18", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages are fluffily furiously ir" }
-{ "l_orderkey": 1254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 36229.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckages boost. furious warhorses cajole" }
-{ "l_orderkey": 2532, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1035.13, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-11-23", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ely final ideas cajole despite the ca" }
-{ "l_orderkey": 2659, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24843.12, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle carefully " }
+{ "l_orderkey": 3844, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es haggle final acco" }
 { "l_orderkey": 4192, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7245.91, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y; excuses use. ironic, close instru" }
+{ "l_orderkey": 4422, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5175.65, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e furiously about t" }
+{ "l_orderkey": 4486, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages. specia" }
 { "l_orderkey": 5063, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18632.34, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "refully quiet reques" }
-{ "l_orderkey": 1317, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35160.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-13", "l_commitdate": "1995-08-08", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "deposits boost thinly blithely final id" }
-{ "l_orderkey": 2023, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "its! carefully ex" }
-{ "l_orderkey": 2055, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16546.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully daringly regular accounts." }
-{ "l_orderkey": 3523, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 49638.24, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " regular requests" }
-{ "l_orderkey": 5798, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 22750.86, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sits poach carefully" }
-{ "l_orderkey": 421, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1034.13, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oldly busy deposit" }
-{ "l_orderkey": 838, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously final ideas. slow, bold " }
-{ "l_orderkey": 1892, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nts. slyly regular asymptot" }
-{ "l_orderkey": 2882, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33092.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. quickly regular e" }
-{ "l_orderkey": 3399, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28955.64, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-29", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "oggedly final theodolites grow. fi" }
-{ "l_orderkey": 3685, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39296.94, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "thely unusual pack" }
-{ "l_orderkey": 3750, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-28", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "usly busy account" }
-{ "l_orderkey": 5636, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 24819.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "counts sleep furiously b" }
-{ "l_orderkey": 5893, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 44467.59, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-09-27", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. regular courts above the carefully silen" }
+{ "l_orderkey": 5219, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2070.26, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " blithely according to the stea" }
+{ "l_orderkey": 5254, "l_partkey": 135, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10351.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-19", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " accounts. silent deposit" }
+{ "l_orderkey": 5572, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47615.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-08", "l_commitdate": "1994-10-14", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully regular platelet" }
+{ "l_orderkey": 5959, "l_partkey": 135, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 50721.37, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usual packages haggle slyly pi" }
 { "l_orderkey": 134, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12409.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly regular pac" }
 { "l_orderkey": 199, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 31023.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-04-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ilent packages doze quickly. thinly " }
+{ "l_orderkey": 421, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1034.13, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-04-27", "l_receiptdate": "1992-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "oldly busy deposit" }
 { "l_orderkey": 518, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12409.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-04-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages thrash slyly" }
+{ "l_orderkey": 644, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47569.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-20", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " special requests was sometimes expre" }
 { "l_orderkey": 773, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9307.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ent orbits haggle fluffily after the " }
+{ "l_orderkey": 838, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-03-25", "l_receiptdate": "1998-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously final ideas. slow, bold " }
+{ "l_orderkey": 1317, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35160.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-13", "l_commitdate": "1995-08-08", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "deposits boost thinly blithely final id" }
 { "l_orderkey": 1408, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43433.46, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "even packages. even accounts cajole" }
 { "l_orderkey": 1666, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32058.03, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-11", "l_receiptdate": "1996-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ding to the express, bold accounts. fu" }
-{ "l_orderkey": 2433, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lithely blithely final ide" }
-{ "l_orderkey": 3332, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27921.51, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ording to the slyly regula" }
-{ "l_orderkey": 5152, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-10", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the final deposits. slyly ironic warth" }
-{ "l_orderkey": 644, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47569.98, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-20", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " special requests was sometimes expre" }
+{ "l_orderkey": 1892, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nts. slyly regular asymptot" }
 { "l_orderkey": 1985, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-11-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "regular requests. furiously express" }
+{ "l_orderkey": 2023, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "its! carefully ex" }
 { "l_orderkey": 2054, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17580.21, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-08-28", "l_receiptdate": "1992-06-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ges nag acc" }
+{ "l_orderkey": 2055, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16546.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-11-12", "l_receiptdate": "1993-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully daringly regular accounts." }
+{ "l_orderkey": 2433, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20682.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lithely blithely final ide" }
 { "l_orderkey": 2853, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26887.38, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-26", "l_commitdate": "1994-06-05", "l_receiptdate": "1994-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "dolphins wake slyly. blith" }
+{ "l_orderkey": 2882, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33092.16, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. quickly regular e" }
 { "l_orderkey": 2976, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13443.69, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-06", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-02-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " furiously final courts boost " }
 { "l_orderkey": 3207, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 33092.16, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l deposits wake beyond the carefully" }
 { "l_orderkey": 3297, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10341.3, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1993-01-21", "l_receiptdate": "1992-12-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ironic idea" }
+{ "l_orderkey": 3332, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27921.51, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ording to the slyly regula" }
+{ "l_orderkey": 3399, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28955.64, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-29", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "oggedly final theodolites grow. fi" }
+{ "l_orderkey": 3523, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 49638.24, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " regular requests" }
+{ "l_orderkey": 3685, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 39296.94, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "thely unusual pack" }
+{ "l_orderkey": 3750, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38262.81, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-28", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "usly busy account" }
 { "l_orderkey": 4039, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44467.59, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-20", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts along the regular in" }
+{ "l_orderkey": 5152, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51706.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-10", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the final deposits. slyly ironic warth" }
 { "l_orderkey": 5477, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32058.03, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special Tiresias cajole furiously. pending" }
+{ "l_orderkey": 5636, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 24819.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "counts sleep furiously b" }
+{ "l_orderkey": 5798, "l_partkey": 134, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 22750.86, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sits poach carefully" }
+{ "l_orderkey": 5893, "l_partkey": 134, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 44467.59, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-02", "l_commitdate": "1992-09-27", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s. regular courts above the carefully silen" }
+{ "l_orderkey": 199, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51656.5, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "essly regular ideas boost sly" }
+{ "l_orderkey": 258, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly blithely special mul" }
 { "l_orderkey": 517, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 11364.43, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-01", "l_receiptdate": "1997-06-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly throughout the fu" }
 { "l_orderkey": 802, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35126.42, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-03-15", "l_receiptdate": "1995-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "instructions cajole carefully. quietl" }
-{ "l_orderkey": 2789, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 43391.46, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ending packages shoul" }
-{ "l_orderkey": 2980, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27894.51, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " theodolites cajole blithely sl" }
-{ "l_orderkey": 3809, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33060.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "xcuses would boost against the fluffily eve" }
-{ "l_orderkey": 4546, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10331.3, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-10-10", "l_receiptdate": "1995-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits alongside of the" }
-{ "l_orderkey": 5120, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28927.64, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-08-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " across the silent requests. caref" }
-{ "l_orderkey": 5414, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15496.95, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-18", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e slyly about the carefully regula" }
-{ "l_orderkey": 199, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51656.5, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "essly regular ideas boost sly" }
+{ "l_orderkey": 1125, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly express packages a" }
 { "l_orderkey": 1284, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40292.07, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "even accoun" }
+{ "l_orderkey": 1472, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26861.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ic packages w" }
 { "l_orderkey": 1506, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47523.98, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sits whithout the blithely ironic packages" }
-{ "l_orderkey": 2791, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45457.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "heodolites use furio" }
-{ "l_orderkey": 3523, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-18", "l_receiptdate": "1998-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts. final accounts detect furiously along " }
-{ "l_orderkey": 4544, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-09-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dolites detect quickly reg" }
-{ "l_orderkey": 4711, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7231.91, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-05-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly. bold accounts use fluff" }
-{ "l_orderkey": 4864, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46490.85, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "round the furiously careful pa" }
 { "l_orderkey": 2467, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7231.91, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-28", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular packages cajole " }
 { "l_orderkey": 2499, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-10-28", "l_receiptdate": "1996-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "to beans across the carefully ironic theodo" }
+{ "l_orderkey": 2789, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 43391.46, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ending packages shoul" }
+{ "l_orderkey": 2791, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45457.72, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-12", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "heodolites use furio" }
+{ "l_orderkey": 2980, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27894.51, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " theodolites cajole blithely sl" }
+{ "l_orderkey": 3040, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9298.17, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ges. pending packages wake. requests" }
 { "l_orderkey": 3140, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28927.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-07-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lar ideas. slyly ironic d" }
 { "l_orderkey": 3458, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49590.24, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-17", "l_commitdate": "1995-01-25", "l_receiptdate": "1995-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously pending dep" }
+{ "l_orderkey": 3523, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-18", "l_receiptdate": "1998-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts. final accounts detect furiously along " }
+{ "l_orderkey": 3809, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 33060.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-03", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "xcuses would boost against the fluffily eve" }
 { "l_orderkey": 4453, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16530.08, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ar excuses nag quickly even accounts. b" }
-{ "l_orderkey": 4964, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 29960.77, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-08-30", "l_receiptdate": "1997-11-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "k accounts nag carefully-- ironic, fin" }
-{ "l_orderkey": 5762, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48557.11, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-03-23", "l_receiptdate": "1997-03-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "equests sleep after the furiously ironic pa" }
-{ "l_orderkey": 258, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-20", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly blithely special mul" }
-{ "l_orderkey": 1125, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4132.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly express packages a" }
-{ "l_orderkey": 1472, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26861.38, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ic packages w" }
-{ "l_orderkey": 3040, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9298.17, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ges. pending packages wake. requests" }
 { "l_orderkey": 4512, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44424.59, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-20", "l_commitdate": "1995-11-28", "l_receiptdate": "1996-01-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "are carefully. theodolites wake" }
-{ "l_orderkey": 384, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14449.82, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckages are slyly after the slyly specia" }
-{ "l_orderkey": 1219, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pecial, ironic requ" }
-{ "l_orderkey": 1668, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8257.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arefully regular tithes! slyl" }
-{ "l_orderkey": 4803, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-05-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular reque" }
-{ "l_orderkey": 5190, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "furiously regular pinto beans. furiously i" }
-{ "l_orderkey": 5696, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 38188.81, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully expres" }
-{ "l_orderkey": 423, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-01", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ccounts. blithely regular pack" }
-{ "l_orderkey": 802, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19610.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "old, furious" }
-{ "l_orderkey": 3111, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31996.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kages detect express attainments" }
-{ "l_orderkey": 3493, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10321.3, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-27", "l_commitdate": "1993-10-07", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hall have to integ" }
-{ "l_orderkey": 5607, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23738.99, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the special, final patterns " }
-{ "l_orderkey": 5957, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29931.77, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sits. final, even asymptotes cajole quickly" }
-{ "l_orderkey": 417, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uriously bol" }
-{ "l_orderkey": 2182, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "en platele" }
-{ "l_orderkey": 2658, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 42317.33, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-07", "l_commitdate": "1995-11-04", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eposits. furiously final theodolite" }
-{ "l_orderkey": 2853, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20642.6, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-06-16", "l_receiptdate": "1994-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e slyly silent foxes. express deposits sno" }
-{ "l_orderkey": 3078, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25803.25, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-05-01", "l_receiptdate": "1993-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "express dinos. carefully ironic" }
-{ "l_orderkey": 3172, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13417.69, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-08-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits haggle along the" }
-{ "l_orderkey": 3397, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 28899.64, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "counts around the final reques" }
-{ "l_orderkey": 3587, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49542.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "beans. blithely final depe" }
-{ "l_orderkey": 5381, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 34060.29, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-09", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly special deposits " }
-{ "l_orderkey": 5958, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33028.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully special theodolites. carefully " }
+{ "l_orderkey": 4544, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 32027.03, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-09-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dolites detect quickly reg" }
+{ "l_orderkey": 4546, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10331.3, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-10-10", "l_receiptdate": "1995-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits alongside of the" }
+{ "l_orderkey": 4711, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7231.91, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-05-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly. bold accounts use fluff" }
+{ "l_orderkey": 4864, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46490.85, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "round the furiously careful pa" }
+{ "l_orderkey": 4964, "l_partkey": 133, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 29960.77, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-08-30", "l_receiptdate": "1997-11-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "k accounts nag carefully-- ironic, fin" }
+{ "l_orderkey": 5120, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28927.64, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-08-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " across the silent requests. caref" }
+{ "l_orderkey": 5414, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15496.95, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-18", "l_commitdate": "1993-06-09", "l_receiptdate": "1993-05-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e slyly about the carefully regula" }
+{ "l_orderkey": 5762, "l_partkey": 133, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 48557.11, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-03-23", "l_receiptdate": "1997-03-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "equests sleep after the furiously ironic pa" }
 { "l_orderkey": 225, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12385.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-04", "l_commitdate": "1995-07-15", "l_receiptdate": "1995-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual requests. bus" }
 { "l_orderkey": 359, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17546.21, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-31", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts according to the blithely" }
+{ "l_orderkey": 384, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14449.82, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ckages are slyly after the slyly specia" }
+{ "l_orderkey": 417, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uriously bol" }
+{ "l_orderkey": 423, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-08-01", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ccounts. blithely regular pack" }
+{ "l_orderkey": 802, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19610.47, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "old, furious" }
 { "l_orderkey": 967, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10321.3, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "old pinto beans alongside of the exp" }
+{ "l_orderkey": 1219, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pecial, ironic requ" }
+{ "l_orderkey": 1668, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8257.04, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-23", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arefully regular tithes! slyl" }
+{ "l_orderkey": 2182, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27867.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "en platele" }
 { "l_orderkey": 2375, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9289.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-17", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly against the packages. bold pinto bean" }
+{ "l_orderkey": 2658, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 42317.33, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-07", "l_commitdate": "1995-11-04", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eposits. furiously final theodolite" }
+{ "l_orderkey": 2853, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20642.6, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-06-16", "l_receiptdate": "1994-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e slyly silent foxes. express deposits sno" }
 { "l_orderkey": 3008, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8257.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1996-01-12", "l_receiptdate": "1995-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly ironic foxes. regular requests h" }
+{ "l_orderkey": 3078, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25803.25, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-05-01", "l_receiptdate": "1993-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "express dinos. carefully ironic" }
+{ "l_orderkey": 3111, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31996.03, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kages detect express attainments" }
+{ "l_orderkey": 3172, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13417.69, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-08-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits haggle along the" }
+{ "l_orderkey": 3397, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 28899.64, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "counts around the final reques" }
+{ "l_orderkey": 3493, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10321.3, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-27", "l_commitdate": "1993-10-07", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hall have to integ" }
+{ "l_orderkey": 3587, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49542.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "beans. blithely final depe" }
 { "l_orderkey": 3814, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 12385.56, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ages cajole. packages haggle. final" }
 { "l_orderkey": 4419, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts wake slyly final dugou" }
+{ "l_orderkey": 4803, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2064.26, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-16", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-05-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular reque" }
+{ "l_orderkey": 5190, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6192.78, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "furiously regular pinto beans. furiously i" }
+{ "l_orderkey": 5381, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 34060.29, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-09", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly special deposits " }
 { "l_orderkey": 5507, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 49542.24, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uriously regular acc" }
-{ "l_orderkey": 515, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-10-08", "l_receiptdate": "1993-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r sauternes boost. final theodolites wake a" }
-{ "l_orderkey": 3264, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35058.42, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "rns haggle carefully. blit" }
-{ "l_orderkey": 4193, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38151.81, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-25", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-05-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "er the quickly regular dependencies wake" }
-{ "l_orderkey": 4544, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41245.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-15", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " detect slyly. evenly pending instru" }
-{ "l_orderkey": 4866, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17529.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-26", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess packages doubt. even somas wake f" }
-{ "l_orderkey": 5441, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-09-22", "l_receiptdate": "1994-10-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ording to the furio" }
-{ "l_orderkey": 5666, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24747.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-03-09", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "on the carefully pending asympto" }
-{ "l_orderkey": 802, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45369.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rmanently idly special requ" }
-{ "l_orderkey": 2407, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40214.07, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-20", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "iously final deposits solv" }
-{ "l_orderkey": 2529, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4124.52, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al dependencies haggle slyly alongsi" }
-{ "l_orderkey": 2627, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggedly final excuses nag packages. f" }
-{ "l_orderkey": 3200, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37120.68, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "f the carefu" }
-{ "l_orderkey": 3521, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-29", "l_commitdate": "1992-12-20", "l_receiptdate": "1993-02-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully duri" }
-{ "l_orderkey": 4292, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-03", "l_commitdate": "1992-02-24", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "bove the silently regula" }
-{ "l_orderkey": 5158, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r requests sleep q" }
-{ "l_orderkey": 5763, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-13", "l_receiptdate": "1998-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ding instruct" }
+{ "l_orderkey": 5607, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23738.99, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-17", "l_commitdate": "1992-02-12", "l_receiptdate": "1992-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the special, final patterns " }
+{ "l_orderkey": 5696, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 38188.81, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-06-23", "l_receiptdate": "1995-08-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully expres" }
+{ "l_orderkey": 5957, "l_partkey": 132, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29931.77, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sits. final, even asymptotes cajole quickly" }
+{ "l_orderkey": 5958, "l_partkey": 132, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 33028.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "e carefully special theodolites. carefully " }
 { "l_orderkey": 225, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3093.39, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-25", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " fluffily about the carefully bold a" }
 { "l_orderkey": 386, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 38151.81, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-03-01", "l_receiptdate": "1995-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ending pearls breach fluffily. slyly pen" }
-{ "l_orderkey": 1793, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uctions; depo" }
-{ "l_orderkey": 3269, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16498.08, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-04-06", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole. silent deposits are f" }
-{ "l_orderkey": 4965, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1993-11-20", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. requests sublate quickly " }
-{ "l_orderkey": 5347, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 21653.73, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly slyly final requests. careful" }
+{ "l_orderkey": 515, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-10-08", "l_receiptdate": "1993-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r sauternes boost. final theodolites wake a" }
+{ "l_orderkey": 802, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45369.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rmanently idly special requ" }
 { "l_orderkey": 994, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 25778.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-03", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual pinto beans." }
 { "l_orderkey": 1061, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 51556.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nding excuses are around the e" }
 { "l_orderkey": 1187, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15466.95, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1993-01-13", "l_receiptdate": "1993-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ests. foxes wake. carefu" }
+{ "l_orderkey": 1793, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uctions; depo" }
 { "l_orderkey": 2116, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "r theodolites use blithely about the ir" }
+{ "l_orderkey": 2407, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 40214.07, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-20", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "iously final deposits solv" }
+{ "l_orderkey": 2529, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4124.52, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-11-18", "l_receiptdate": "1996-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al dependencies haggle slyly alongsi" }
+{ "l_orderkey": 2627, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggedly final excuses nag packages. f" }
 { "l_orderkey": 2755, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5155.65, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-27", "l_commitdate": "1992-04-07", "l_receiptdate": "1992-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e the furi" }
+{ "l_orderkey": 3200, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37120.68, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "f the carefu" }
+{ "l_orderkey": 3264, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 35058.42, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "rns haggle carefully. blit" }
+{ "l_orderkey": 3269, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16498.08, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-04-06", "l_receiptdate": "1996-03-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole. silent deposits are f" }
 { "l_orderkey": 3328, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25778.25, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e unusual, r" }
+{ "l_orderkey": 3521, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-29", "l_commitdate": "1992-12-20", "l_receiptdate": "1993-02-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully duri" }
 { "l_orderkey": 3749, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7217.91, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-04-05", "l_receiptdate": "1995-04-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he slyly ironic packages" }
 { "l_orderkey": 3814, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7217.91, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es sleep furiou" }
+{ "l_orderkey": 4193, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 38151.81, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-25", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-05-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "er the quickly regular dependencies wake" }
+{ "l_orderkey": 4292, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6186.78, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-03", "l_commitdate": "1992-02-24", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "bove the silently regula" }
+{ "l_orderkey": 4544, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 41245.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-15", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " detect slyly. evenly pending instru" }
+{ "l_orderkey": 4866, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17529.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-26", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess packages doubt. even somas wake f" }
+{ "l_orderkey": 4965, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28871.64, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1993-11-20", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. requests sublate quickly " }
+{ "l_orderkey": 5158, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r requests sleep q" }
 { "l_orderkey": 5255, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2062.26, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-27", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ajole blithely fluf" }
+{ "l_orderkey": 5347, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 21653.73, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sly slyly final requests. careful" }
+{ "l_orderkey": 5441, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50525.37, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-09-22", "l_receiptdate": "1994-10-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ording to the furio" }
 { "l_orderkey": 5445, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12373.56, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-02", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly pending pinto beans was slyly al" }
+{ "l_orderkey": 5666, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24747.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-03-09", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "on the carefully pending asympto" }
 { "l_orderkey": 5702, "l_partkey": 131, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45369.72, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-28", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-12-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ake according to th" }
+{ "l_orderkey": 5763, "l_partkey": 131, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32996.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-13", "l_receiptdate": "1998-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ding instruct" }
+{ "l_orderkey": 256, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46355.85, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " grouches. ideas wake quickly ar" }
 { "l_orderkey": 451, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37084.68, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "rges can haggle carefully ironic, dogged " }
+{ "l_orderkey": 583, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express req" }
 { "l_orderkey": 644, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11331.43, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ealthy pinto beans use carefu" }
+{ "l_orderkey": 864, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-23", "l_receiptdate": "1998-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gside of the furiously special" }
+{ "l_orderkey": 1447, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23692.99, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-25", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " dazzle quickly deposits. f" }
+{ "l_orderkey": 1538, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37084.68, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-11", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al deposits mo" }
 { "l_orderkey": 1856, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 43265.46, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-23", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usly final deposits" }
 { "l_orderkey": 2051, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49446.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-04", "l_commitdate": "1996-06-14", "l_receiptdate": "1996-05-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts. pending platelets believe about" }
+{ "l_orderkey": 2179, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22662.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lphins cajole acr" }
 { "l_orderkey": 2434, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28843.64, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-28", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ven theodolites around the slyly" }
+{ "l_orderkey": 2499, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6180.78, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-12-14", "l_receiptdate": "1995-12-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cording to the" }
+{ "l_orderkey": 2886, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47385.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ously final packages sleep blithely regular" }
+{ "l_orderkey": 2950, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32964.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-21", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its wake carefully slyly final ideas." }
+{ "l_orderkey": 3009, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26783.38, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-06-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uriously specia" }
 { "l_orderkey": 3459, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30903.9, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-22", "l_commitdate": "1994-09-12", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nic theodolites; evenly i" }
+{ "l_orderkey": 3460, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 48416.11, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es haggle slyly regular accounts. fi" }
+{ "l_orderkey": 3522, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7210.91, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e stealthil" }
 { "l_orderkey": 3713, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14421.82, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "eposits impress according" }
+{ "l_orderkey": 3782, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41205.2, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s instructions. regular accou" }
 { "l_orderkey": 3815, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11331.43, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1997-11-05", "l_receiptdate": "1997-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sleep blithe" }
 { "l_orderkey": 4386, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10301.3, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "gainst the quickly expre" }
 { "l_orderkey": 4386, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 40175.07, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "structions cajole quickly express" }
-{ "l_orderkey": 256, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46355.85, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " grouches. ideas wake quickly ar" }
-{ "l_orderkey": 583, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express req" }
-{ "l_orderkey": 1447, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 23692.99, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1992-12-25", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " dazzle quickly deposits. f" }
-{ "l_orderkey": 2499, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6180.78, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-12-14", "l_receiptdate": "1995-12-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "cording to the" }
-{ "l_orderkey": 2950, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32964.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-21", "l_commitdate": "1997-08-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its wake carefully slyly final ideas." }
-{ "l_orderkey": 3009, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26783.38, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-06-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uriously specia" }
-{ "l_orderkey": 4774, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 30903.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "tions against the blithely final theodolit" }
-{ "l_orderkey": 5255, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42235.33, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tect blithely against t" }
-{ "l_orderkey": 864, "l_partkey": 130, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 35024.42, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-23", "l_receiptdate": "1998-01-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gside of the furiously special" }
-{ "l_orderkey": 1538, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 37084.68, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-11", "l_commitdate": "1995-09-10", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al deposits mo" }
-{ "l_orderkey": 2179, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22662.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lphins cajole acr" }
-{ "l_orderkey": 3460, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 48416.11, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es haggle slyly regular accounts. fi" }
-{ "l_orderkey": 3522, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7210.91, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e stealthil" }
 { "l_orderkey": 4705, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31934.03, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-03", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "furiously final accou" }
+{ "l_orderkey": 4774, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 30903.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "tions against the blithely final theodolit" }
 { "l_orderkey": 4994, "l_partkey": 130, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 31934.03, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-14", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lar decoys cajole fluffil" }
-{ "l_orderkey": 2886, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47385.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-02", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ously final packages sleep blithely regular" }
-{ "l_orderkey": 3782, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41205.2, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-16", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s instructions. regular accou" }
 { "l_orderkey": 5223, "l_partkey": 130, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 41205.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-01", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly pending " }
+{ "l_orderkey": 5255, "l_partkey": 130, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42235.33, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tect blithely against t" }
 { "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
-{ "l_orderkey": 1318, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31902.72, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ve the carefully expr" }
-{ "l_orderkey": 2274, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-22", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " express packages. even accounts hagg" }
-{ "l_orderkey": 3300, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3087.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-02", "l_receiptdate": "1995-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "g according to the dugouts. caref" }
-{ "l_orderkey": 3557, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38077.44, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-16", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gside of the ca" }
-{ "l_orderkey": 3749, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9262.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses cajole blithely pla" }
-{ "l_orderkey": 4197, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51456.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-11-01", "l_receiptdate": "1996-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully bold asymptotes nag blithe" }
-{ "l_orderkey": 5637, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "oss the carefully express warhorses" }
-{ "l_orderkey": 5699, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "rmanent packages sleep across the f" }
-{ "l_orderkey": 929, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-06", "l_receiptdate": "1993-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ges haggle careful" }
-{ "l_orderkey": 1280, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17495.04, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-04", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-02-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ructions integrate across the th" }
-{ "l_orderkey": 2723, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41164.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-11-22", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unwind fluffily carefully regular realms." }
-{ "l_orderkey": 4069, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven theodolites nag quickly. fluffi" }
-{ "l_orderkey": 4993, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "pending, regular requests solve caref" }
-{ "l_orderkey": 5090, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47339.52, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lose theodolites sleep blit" }
-{ "l_orderkey": 5411, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15436.8, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "attainments sleep slyly ironic" }
-{ "l_orderkey": 5671, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13378.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-03-26", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ers according to the ironic, unusual excu" }
-{ "l_orderkey": 2149, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-05-11", "l_receiptdate": "1993-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uriously final pac" }
-{ "l_orderkey": 2177, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-02-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". theodolites haggle carefu" }
-{ "l_orderkey": 4647, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34990.08, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly sly accounts" }
 { "l_orderkey": 130, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14407.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-15", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-09-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. final instruction" }
 { "l_orderkey": 229, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29844.48, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s, final request" }
 { "l_orderkey": 290, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2058.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". permanently furious reques" }
+{ "l_orderkey": 929, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-06", "l_receiptdate": "1993-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ges haggle careful" }
 { "l_orderkey": 995, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28815.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "pades. quick, final frays use flu" }
 { "l_orderkey": 1219, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4116.48, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-24", "l_commitdate": "1995-11-22", "l_receiptdate": "1995-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lly quick requests. blithely even h" }
+{ "l_orderkey": 1280, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17495.04, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-04", "l_commitdate": "1993-04-10", "l_receiptdate": "1993-02-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ructions integrate across the th" }
+{ "l_orderkey": 1318, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31902.72, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-08-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ve the carefully expr" }
 { "l_orderkey": 2018, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23669.76, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ingly even theodolites s" }
 { "l_orderkey": 2022, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20582.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-31", "l_commitdate": "1992-04-17", "l_receiptdate": "1992-04-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "r deposits kindle " }
+{ "l_orderkey": 2149, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-05", "l_commitdate": "1993-05-11", "l_receiptdate": "1993-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uriously final pac" }
+{ "l_orderkey": 2177, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-02-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". theodolites haggle carefu" }
+{ "l_orderkey": 2274, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18524.16, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-11-22", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " express packages. even accounts hagg" }
+{ "l_orderkey": 2723, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 41164.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-11-22", "l_receiptdate": "1995-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unwind fluffily carefully regular realms." }
 { "l_orderkey": 3175, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12349.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-10-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ter the pending deposits. slyly e" }
+{ "l_orderkey": 3300, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3087.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-10-02", "l_receiptdate": "1995-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "g according to the dugouts. caref" }
 { "l_orderkey": 3462, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6174.72, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-06-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "iously regular fo" }
+{ "l_orderkey": 3557, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 38077.44, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-16", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gside of the ca" }
+{ "l_orderkey": 3749, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9262.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uses cajole blithely pla" }
+{ "l_orderkey": 4069, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ven theodolites nag quickly. fluffi" }
+{ "l_orderkey": 4197, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 51456.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-11-01", "l_receiptdate": "1996-11-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully bold asymptotes nag blithe" }
+{ "l_orderkey": 4647, "l_partkey": 129, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34990.08, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly sly accounts" }
+{ "l_orderkey": 4993, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "pending, regular requests solve caref" }
 { "l_orderkey": 5063, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31902.72, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-06-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages. ironic, ironic courts wake. carefu" }
+{ "l_orderkey": 5090, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47339.52, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lose theodolites sleep blit" }
 { "l_orderkey": 5350, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-25", "l_commitdate": "1993-12-27", "l_receiptdate": "1993-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es. blithe theodolites haggl" }
+{ "l_orderkey": 5411, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15436.8, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-07-15", "l_receiptdate": "1997-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "attainments sleep slyly ironic" }
 { "l_orderkey": 5543, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 40135.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l excuses are furiously. slyly unusual requ" }
+{ "l_orderkey": 5637, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27786.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "oss the carefully express warhorses" }
 { "l_orderkey": 5671, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47339.52, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lar pinto beans detect care" }
+{ "l_orderkey": 5671, "l_partkey": 129, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13378.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-03-26", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ers according to the ironic, unusual excu" }
+{ "l_orderkey": 5699, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 46310.4, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "rmanent packages sleep across the f" }
 { "l_orderkey": 5829, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6174.72, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-22", "l_commitdate": "1997-03-12", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sts. slyly special fo" }
 { "l_orderkey": 5953, "l_partkey": 129, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 37048.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-06-24", "l_receiptdate": "1992-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " cajole furio" }
-{ "l_orderkey": 871, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1996-01-12", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar ideas-- slyly even accou" }
-{ "l_orderkey": 2593, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6168.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-28", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ular packages. re" }
-{ "l_orderkey": 4996, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12337.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-22", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usly bold requests sleep dogge" }
-{ "l_orderkey": 1953, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ular, regular i" }
-{ "l_orderkey": 2566, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1028.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "theodolites wake pending" }
-{ "l_orderkey": 2629, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29815.48, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eposits serve unusual, express i" }
-{ "l_orderkey": 3168, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13365.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-05", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ironic somas haggle quick" }
-{ "l_orderkey": 3527, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17478.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular instruction" }
-{ "l_orderkey": 4804, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45237.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "aggle quickly among the slyly fi" }
-{ "l_orderkey": 1028, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2056.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s alongside of the regular asymptotes sleep" }
-{ "l_orderkey": 1668, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-08", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "even platelets across the silent " }
-{ "l_orderkey": 2503, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40096.68, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d carefully fluffily" }
-{ "l_orderkey": 2662, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ajole carefully. sp" }
-{ "l_orderkey": 3396, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34956.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-30", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": ". slyly unusual packages wak" }
-{ "l_orderkey": 3650, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44209.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gside of the quick" }
-{ "l_orderkey": 4096, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20562.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-13", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sual requests. furiously bold packages wake" }
-{ "l_orderkey": 4257, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33927.96, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uffily regular accounts ar" }
-{ "l_orderkey": 4710, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48321.64, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely express packages. even, ironic re" }
 { "l_orderkey": 388, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 47293.52, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-01-26", "l_receiptdate": "1993-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans nag about the careful reque" }
 { "l_orderkey": 390, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13365.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sleep carefully idle packages. blithely " }
 { "l_orderkey": 711, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47293.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1993-11-19", "l_receiptdate": "1994-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "deposits. permanen" }
 { "l_orderkey": 711, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20562.4, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-17", "l_commitdate": "1993-11-10", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "kly regular acco" }
+{ "l_orderkey": 871, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1996-01-12", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar ideas-- slyly even accou" }
+{ "l_orderkey": 1028, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2056.24, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-10", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s alongside of the regular asymptotes sleep" }
 { "l_orderkey": 1538, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 43181.04, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-10", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests cajole blithely " }
+{ "l_orderkey": 1668, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-08", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "even platelets across the silent " }
+{ "l_orderkey": 1953, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ular, regular i" }
+{ "l_orderkey": 2503, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 40096.68, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d carefully fluffily" }
+{ "l_orderkey": 2566, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1028.12, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "theodolites wake pending" }
+{ "l_orderkey": 2593, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6168.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-28", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ular packages. re" }
+{ "l_orderkey": 2629, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29815.48, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eposits serve unusual, express i" }
+{ "l_orderkey": 2662, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ajole carefully. sp" }
+{ "l_orderkey": 3168, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13365.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-05", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ironic somas haggle quick" }
+{ "l_orderkey": 3396, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34956.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-30", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": ". slyly unusual packages wak" }
+{ "l_orderkey": 3527, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17478.04, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular instruction" }
+{ "l_orderkey": 3650, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44209.16, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gside of the quick" }
+{ "l_orderkey": 4096, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20562.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-13", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sual requests. furiously bold packages wake" }
+{ "l_orderkey": 4257, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33927.96, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uffily regular accounts ar" }
 { "l_orderkey": 4614, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 42152.92, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ackages haggle carefully about the even, b" }
 { "l_orderkey": 4677, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25703.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "unts doubt furiousl" }
+{ "l_orderkey": 4710, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48321.64, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely express packages. even, ironic re" }
+{ "l_orderkey": 4804, "l_partkey": 128, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 45237.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "aggle quickly among the slyly fi" }
+{ "l_orderkey": 4996, "l_partkey": 128, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12337.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-22", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "usly bold requests sleep dogge" }
 { "l_orderkey": 5185, "l_partkey": 128, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8224.96, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-02", "l_receiptdate": "1997-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sts around the slyly perma" }
 { "l_orderkey": 5511, "l_partkey": 128, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 50377.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-21", "l_commitdate": "1995-01-27", "l_receiptdate": "1994-12-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "bout the requests. theodolites " }
+{ "l_orderkey": 37, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the final requests. ca" }
+{ "l_orderkey": 163, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12325.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly blithe accounts cajole " }
+{ "l_orderkey": 646, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t blithely regular deposits. quic" }
 { "l_orderkey": 1316, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ges haggle of the" }
 { "l_orderkey": 1671, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12325.44, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-17", "l_commitdate": "1996-09-02", "l_receiptdate": "1996-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "special, ironic" }
+{ "l_orderkey": 1827, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4108.48, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-10", "l_receiptdate": "1996-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. blithely" }
+{ "l_orderkey": 2023, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9244.08, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-06-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular pinto beans poa" }
+{ "l_orderkey": 2087, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "the quickly idle acco" }
+{ "l_orderkey": 2243, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10271.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-26", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "express, daring foxes affix fur" }
+{ "l_orderkey": 2434, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r deposits sleep furiou" }
+{ "l_orderkey": 2469, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8216.96, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-20", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. regular" }
+{ "l_orderkey": 2945, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28759.36, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le slyly along the eve" }
+{ "l_orderkey": 2978, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 43139.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial requests nag blithely alongside of th" }
+{ "l_orderkey": 3392, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34922.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-20", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e carefully even braids. " }
 { "l_orderkey": 3394, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30813.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-12", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t ideas according to the fluffily iro" }
+{ "l_orderkey": 3588, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22596.64, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "inal accounts. pending, bo" }
+{ "l_orderkey": 3780, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25678.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-07-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l, unusual " }
 { "l_orderkey": 3808, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48274.64, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fully for the quickly final deposits: flu" }
+{ "l_orderkey": 3876, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42111.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-10-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " quickly blit" }
 { "l_orderkey": 4036, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20542.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-09-03", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly bold deposits cajole pending, blithe" }
 { "l_orderkey": 4870, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6162.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-09", "l_commitdate": "1994-10-16", "l_receiptdate": "1994-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ress requests. bold, silent pinto bea" }
 { "l_orderkey": 5281, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23623.76, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1996-01-26", "l_receiptdate": "1996-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". final theodolites cajole. ironic p" }
 { "l_orderkey": 5606, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-11", "l_commitdate": "1997-01-13", "l_receiptdate": "1997-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ter the ironic accounts. even, ironic depos" }
 { "l_orderkey": 5798, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2054.24, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-25", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e furiously across " }
-{ "l_orderkey": 2087, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-27", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "the quickly idle acco" }
-{ "l_orderkey": 2434, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "r deposits sleep furiou" }
-{ "l_orderkey": 2945, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28759.36, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "le slyly along the eve" }
-{ "l_orderkey": 163, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12325.44, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly blithe accounts cajole " }
-{ "l_orderkey": 646, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1027.12, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t blithely regular deposits. quic" }
-{ "l_orderkey": 2243, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10271.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-26", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "express, daring foxes affix fur" }
-{ "l_orderkey": 2469, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8216.96, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-20", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "s. regular" }
-{ "l_orderkey": 2978, "l_partkey": 127, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 43139.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial requests nag blithely alongside of th" }
-{ "l_orderkey": 3392, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34922.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-20", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e carefully even braids. " }
-{ "l_orderkey": 3588, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22596.64, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-08", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "inal accounts. pending, bo" }
-{ "l_orderkey": 3780, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25678.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-27", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-07-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l, unusual " }
-{ "l_orderkey": 3876, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 42111.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-10-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " quickly blit" }
-{ "l_orderkey": 37, "l_partkey": 127, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 40057.68, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the final requests. ca" }
-{ "l_orderkey": 1827, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4108.48, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-09-10", "l_receiptdate": "1996-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. blithely" }
-{ "l_orderkey": 2023, "l_partkey": 127, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9244.08, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-06-30", "l_receiptdate": "1992-06-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular pinto beans poa" }
 { "l_orderkey": 5926, "l_partkey": 127, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47247.52, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts integrate. courts haggl" }
-{ "l_orderkey": 448, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts thrash quickly among the b" }
-{ "l_orderkey": 1793, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nic foxes along the even" }
-{ "l_orderkey": 3396, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27705.24, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-07-26", "l_receiptdate": "1994-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " theodolites " }
-{ "l_orderkey": 3555, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9235.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "are. slyly final foxes acro" }
-{ "l_orderkey": 3879, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46175.4, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-18", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly according to the expr" }
 { "l_orderkey": 164, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38992.56, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-04", "l_commitdate": "1992-11-23", "l_receiptdate": "1993-01-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts cajole fluffily regular packages. b" }
+{ "l_orderkey": 448, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-10-20", "l_receiptdate": "1995-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts thrash quickly among the b" }
+{ "l_orderkey": 641, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18470.16, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p blithely bold packages. quick" }
 { "l_orderkey": 804, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ehind the quietly regular pac" }
 { "l_orderkey": 897, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13339.56, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-17", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "bold accounts mold carefully! braids" }
+{ "l_orderkey": 1024, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "des the slyly even" }
 { "l_orderkey": 1027, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2052.24, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-09-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. quickly unusual waters inside " }
-{ "l_orderkey": 2752, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22574.64, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-20", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "equests nag. regular dependencies are furio" }
-{ "l_orderkey": 2820, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33861.96, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "carefully even pinto beans. " }
-{ "l_orderkey": 3365, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24626.88, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-01-09", "l_receiptdate": "1995-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans? carefully regula" }
-{ "l_orderkey": 3684, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-09-02", "l_receiptdate": "1993-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its boost alongside" }
-{ "l_orderkey": 3907, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests according to the slyly pending " }
-{ "l_orderkey": 5413, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1997-11-20", "l_receiptdate": "1998-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " theodolites. furiously ironic instr" }
-{ "l_orderkey": 641, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18470.16, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p blithely bold packages. quick" }
 { "l_orderkey": 1184, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3078.36, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1997-12-19", "l_receiptdate": "1998-02-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ar packages. final packages cajol" }
+{ "l_orderkey": 1728, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1026.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly. carefully ex" }
+{ "l_orderkey": 1793, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4104.48, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nic foxes along the even" }
 { "l_orderkey": 2049, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-02-04", "l_receiptdate": "1995-12-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ial accounts are among the furiously perma" }
+{ "l_orderkey": 2375, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20522.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ckages! blithely enticing deposi" }
 { "l_orderkey": 2533, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38992.56, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "of the regular accounts. even packages caj" }
 { "l_orderkey": 2594, "l_partkey": 126, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24626.88, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-02-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar accounts sleep fur" }
+{ "l_orderkey": 2752, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22574.64, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-20", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "equests nag. regular dependencies are furio" }
+{ "l_orderkey": 2820, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33861.96, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "carefully even pinto beans. " }
 { "l_orderkey": 3040, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30783.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-06", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-08-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "x furiously bold packages. expres" }
+{ "l_orderkey": 3365, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24626.88, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-01-09", "l_receiptdate": "1995-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans? carefully regula" }
+{ "l_orderkey": 3396, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27705.24, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-07-26", "l_receiptdate": "1994-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " theodolites " }
 { "l_orderkey": 3492, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7182.84, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-10", "l_commitdate": "1995-01-03", "l_receiptdate": "1995-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thely regular dolphi" }
+{ "l_orderkey": 3555, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9235.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-13", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "are. slyly final foxes acro" }
+{ "l_orderkey": 3684, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-20", "l_commitdate": "1993-09-02", "l_receiptdate": "1993-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "its boost alongside" }
 { "l_orderkey": 3747, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 32835.84, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests shall h" }
-{ "l_orderkey": 5027, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32835.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-29", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cording to" }
-{ "l_orderkey": 5700, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23600.76, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-01-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly carefully fluffy hockey" }
-{ "l_orderkey": 1024, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-06", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "des the slyly even" }
-{ "l_orderkey": 1728, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1026.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly. carefully ex" }
-{ "l_orderkey": 2375, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20522.4, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ckages! blithely enticing deposi" }
+{ "l_orderkey": 3879, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 46175.4, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-18", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly according to the expr" }
+{ "l_orderkey": 3907, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 34888.08, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-06", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests according to the slyly pending " }
 { "l_orderkey": 4004, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 20522.4, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-06-14", "l_receiptdate": "1993-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". ironic deposits cajole blithely?" }
 { "l_orderkey": 4358, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 48227.64, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "refully busy dep" }
 { "l_orderkey": 4614, "l_partkey": 126, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6156.72, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-07-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ake quickly quickly regular epitap" }
+{ "l_orderkey": 5027, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32835.84, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-29", "l_receiptdate": "1997-11-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cording to" }
+{ "l_orderkey": 5413, "l_partkey": 126, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 49253.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1997-11-20", "l_receiptdate": "1998-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " theodolites. furiously ironic instr" }
+{ "l_orderkey": 5700, "l_partkey": 126, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23600.76, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-01-31", "l_receiptdate": "1998-01-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly carefully fluffy hockey" }
 { "l_orderkey": 356, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37929.44, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-08-24", "l_receiptdate": "1994-08-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ndencies are since the packag" }
-{ "l_orderkey": 2848, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34854.08, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-15", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ts along the blithely regu" }
-{ "l_orderkey": 4069, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3075.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake furiously! slyl" }
-{ "l_orderkey": 5124, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45105.28, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-13", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. carefully unusual d" }
-{ "l_orderkey": 5314, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16401.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "hely unusual packages acc" }
-{ "l_orderkey": 1795, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 32803.84, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " asymptotes across the bold," }
-{ "l_orderkey": 4288, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7175.84, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-15", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ngside of the special platelet" }
 { "l_orderkey": 390, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11276.32, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "according to the foxes are furiously " }
+{ "l_orderkey": 868, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19477.28, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ely even deposits lose blithe" }
 { "l_orderkey": 1346, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49205.76, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-28", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " along the carefully spec" }
 { "l_orderkey": 1477, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 43055.04, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-02", "l_commitdate": "1997-11-02", "l_receiptdate": "1997-11-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lithely after the ir" }
+{ "l_orderkey": 1795, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 32803.84, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " asymptotes across the bold," }
 { "l_orderkey": 2690, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 46130.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-06-02", "l_receiptdate": "1996-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ounts. slyly regular dependencies wa" }
+{ "l_orderkey": 2848, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34854.08, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-15", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ts along the blithely regu" }
+{ "l_orderkey": 2883, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27678.24, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. brave pinto beans nag furiously" }
 { "l_orderkey": 3264, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11276.32, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "regular packages" }
 { "l_orderkey": 3393, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 45105.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ld requests hag" }
-{ "l_orderkey": 4230, "l_partkey": 125, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 51256.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. final instructions in" }
-{ "l_orderkey": 5925, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31778.72, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-02", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly. furiously regular deposi" }
-{ "l_orderkey": 868, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19477.28, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ely even deposits lose blithe" }
-{ "l_orderkey": 2883, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27678.24, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. brave pinto beans nag furiously" }
+{ "l_orderkey": 4069, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3075.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake furiously! slyl" }
 { "l_orderkey": 4131, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30753.6, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-04-13", "l_receiptdate": "1998-04-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he fluffily express depen" }
+{ "l_orderkey": 4230, "l_partkey": 125, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 51256.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-29", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. final instructions in" }
+{ "l_orderkey": 4288, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7175.84, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-15", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ngside of the special platelet" }
 { "l_orderkey": 4291, "l_partkey": 125, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 44080.16, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. quietly regular " }
+{ "l_orderkey": 5124, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 45105.28, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-13", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. carefully unusual d" }
 { "l_orderkey": 5218, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33828.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-09-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ronic instructi" }
+{ "l_orderkey": 5314, "l_partkey": 125, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16401.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-25", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "hely unusual packages acc" }
+{ "l_orderkey": 5925, "l_partkey": 125, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31778.72, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-02", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly. furiously regular deposi" }
+{ "l_orderkey": 5, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26627.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sts use slyly quickly special instruc" }
+{ "l_orderkey": 96, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23554.76, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep-- carefully reg" }
+{ "l_orderkey": 99, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5120.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests cajole fluffily waters. blithe" }
 { "l_orderkey": 290, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23554.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-02-21", "l_receiptdate": "1994-04-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully unusual packages. " }
-{ "l_orderkey": 1671, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11265.32, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-18", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes sleep blithely" }
-{ "l_orderkey": 2306, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uld have to mold. s" }
-{ "l_orderkey": 2465, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20482.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uriously? furiously ironic excu" }
-{ "l_orderkey": 5765, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nic requests. deposits wake quickly among " }
 { "l_orderkey": 390, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17410.04, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ending, pending pinto beans wake slyl" }
 { "l_orderkey": 420, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 40964.8, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " after the special" }
 { "l_orderkey": 708, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3072.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly pending foxes. " }
 { "l_orderkey": 1088, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3072.36, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-08-02", "l_receiptdate": "1992-06-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "pecial theodolites " }
-{ "l_orderkey": 5089, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47109.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "above the express accounts. exc" }
-{ "l_orderkey": 5798, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14337.68, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he special, bold packages. carefully iron" }
-{ "l_orderkey": 5, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26627.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sts use slyly quickly special instruc" }
-{ "l_orderkey": 99, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5120.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests cajole fluffily waters. blithe" }
 { "l_orderkey": 1283, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 44037.16, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-29", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "requests sleep slyly about the " }
 { "l_orderkey": 1346, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6144.72, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-13", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-09-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "inst the furiously final theodolites. caref" }
 { "l_orderkey": 1378, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18434.16, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-19", "l_commitdate": "1996-05-16", "l_receiptdate": "1996-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " theodolites. i" }
+{ "l_orderkey": 1671, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11265.32, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-09-18", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes sleep blithely" }
 { "l_orderkey": 1697, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27651.24, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly regular packages across the silent, b" }
-{ "l_orderkey": 1731, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 41988.92, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-05", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle across the blithely ironi" }
-{ "l_orderkey": 2722, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21506.52, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully around the furiously ironic pac" }
-{ "l_orderkey": 5159, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39940.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-08", "l_receiptdate": "1997-01-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re furiously after the pending dolphin" }
-{ "l_orderkey": 5223, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25603.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y express ideas impress" }
-{ "l_orderkey": 96, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23554.76, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep-- carefully reg" }
 { "l_orderkey": 1703, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 49157.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ggle slyly furiously regular theodol" }
+{ "l_orderkey": 1731, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 41988.92, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-05", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "haggle across the blithely ironi" }
 { "l_orderkey": 1985, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 43013.04, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " patterns? final requests after the sp" }
 { "l_orderkey": 2209, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24578.88, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-09", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " along the bol" }
+{ "l_orderkey": 2306, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uld have to mold. s" }
+{ "l_orderkey": 2465, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20482.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uriously? furiously ironic excu" }
 { "l_orderkey": 2594, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13313.56, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully special accounts use courts" }
 { "l_orderkey": 2629, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31747.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-24", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ate blithely bold, regular deposits. bold" }
+{ "l_orderkey": 2722, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21506.52, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully around the furiously ironic pac" }
 { "l_orderkey": 3392, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7168.84, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-07", "l_commitdate": "1996-01-09", "l_receiptdate": "1995-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "as. express, final accounts dou" }
 { "l_orderkey": 3587, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31747.72, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "press fluffily regul" }
+{ "l_orderkey": 5089, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 47109.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "above the express accounts. exc" }
+{ "l_orderkey": 5159, "l_partkey": 124, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39940.68, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-08", "l_receiptdate": "1997-01-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re furiously after the pending dolphin" }
 { "l_orderkey": 5184, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19458.28, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-15", "l_commitdate": "1998-10-12", "l_receiptdate": "1998-11-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "refully express platelets sleep carefull" }
+{ "l_orderkey": 5223, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25603.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-08-13", "l_receiptdate": "1994-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y express ideas impress" }
 { "l_orderkey": 5347, "l_partkey": 124, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48133.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-03-29", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "across the slyly bol" }
 { "l_orderkey": 5696, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19458.28, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-31", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "unusual requests sleep furiously ru" }
+{ "l_orderkey": 5765, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29699.48, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nic requests. deposits wake quickly among " }
+{ "l_orderkey": 5798, "l_partkey": 124, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14337.68, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he special, bold packages. carefully iron" }
 { "l_orderkey": 291, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21485.52, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-05-10", "l_receiptdate": "1994-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y quickly regular theodolites. final t" }
+{ "l_orderkey": 1059, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly regular theodo" }
+{ "l_orderkey": 1474, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30693.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly. evenly express " }
+{ "l_orderkey": 1505, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51156.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly special platelets. requests ar" }
+{ "l_orderkey": 1607, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-02-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uches cajole. accounts ar" }
+{ "l_orderkey": 2113, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40924.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1997-12-11", "l_receiptdate": "1998-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "bout the quickly ironic t" }
+{ "l_orderkey": 2242, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15346.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "its. carefully express packages cajole. bli" }
+{ "l_orderkey": 2503, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33762.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nal courts integrate according to the" }
+{ "l_orderkey": 2823, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20462.4, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-12-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "its sleep between the unusual, ironic pac" }
 { "l_orderkey": 2913, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-09-27", "l_receiptdate": "1997-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". final packages a" }
+{ "l_orderkey": 3011, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42971.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "osits haggle quickly pending, " }
 { "l_orderkey": 3329, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1023.12, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-22", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "regular packages are carefull" }
+{ "l_orderkey": 3392, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1996-01-17", "l_receiptdate": "1995-12-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the fluffily bold deposits." }
 { "l_orderkey": 3586, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 33762.96, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-03-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "iously regular pinto beans integrate" }
 { "l_orderkey": 3813, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-30", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y ideas. final ideas about the sp" }
 { "l_orderkey": 3877, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 7161.84, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-14", "l_commitdate": "1993-07-09", "l_receiptdate": "1993-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar dolphins cajole silently " }
-{ "l_orderkey": 4967, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1023.12, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits. unusual frets thrash furiously" }
-{ "l_orderkey": 5763, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8184.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-23", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "foxes wake slyly. car" }
-{ "l_orderkey": 1474, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30693.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly. evenly express " }
-{ "l_orderkey": 2113, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40924.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-16", "l_commitdate": "1997-12-11", "l_receiptdate": "1998-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "bout the quickly ironic t" }
-{ "l_orderkey": 2242, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15346.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-09-21", "l_receiptdate": "1997-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "its. carefully express packages cajole. bli" }
-{ "l_orderkey": 2823, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20462.4, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-12-06", "l_receiptdate": "1995-12-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "its sleep between the unusual, ironic pac" }
+{ "l_orderkey": 3941, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits haggle furiously even" }
 { "l_orderkey": 4065, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 29670.48, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-08-19", "l_receiptdate": "1994-07-17", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "equests. packages sleep slyl" }
+{ "l_orderkey": 4455, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34786.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " slyly ironic requests. quickly even d" }
 { "l_orderkey": 4548, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 48086.64, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-24", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. excuses use slyly spec" }
 { "l_orderkey": 4673, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 9208.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ages nag across " }
-{ "l_orderkey": 5095, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28647.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-06-27", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " into the final courts. ca" }
-{ "l_orderkey": 5414, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49109.76, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " silent dolphins; fluffily regular tithe" }
-{ "l_orderkey": 5760, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits nag. even, regular ideas cajole b" }
-{ "l_orderkey": 1059, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-12", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly regular theodo" }
-{ "l_orderkey": 1505, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 51156.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly special platelets. requests ar" }
-{ "l_orderkey": 3011, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42971.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "osits haggle quickly pending, " }
-{ "l_orderkey": 3941, "l_partkey": 123, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits haggle furiously even" }
-{ "l_orderkey": 4455, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34786.08, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-24", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " slyly ironic requests. quickly even d" }
-{ "l_orderkey": 1607, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39901.68, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-02-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uches cajole. accounts ar" }
-{ "l_orderkey": 2503, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33762.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nal courts integrate according to the" }
-{ "l_orderkey": 3392, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13300.56, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1996-01-17", "l_receiptdate": "1995-12-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the fluffily bold deposits." }
+{ "l_orderkey": 4967, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 1023.12, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits. unusual frets thrash furiously" }
 { "l_orderkey": 5024, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42971.04, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-02", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate. busily spec" }
 { "l_orderkey": 5059, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " special ideas poach blithely qu" }
+{ "l_orderkey": 5095, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28647.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-06-27", "l_receiptdate": "1992-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " into the final courts. ca" }
 { "l_orderkey": 5189, "l_partkey": 123, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 14323.68, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-23", "l_commitdate": "1994-01-05", "l_receiptdate": "1994-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unusual packag" }
+{ "l_orderkey": 5414, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 49109.76, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " silent dolphins; fluffily regular tithe" }
 { "l_orderkey": 5700, "l_partkey": 123, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30693.6, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly blithely final instructions. fl" }
+{ "l_orderkey": 5760, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19439.28, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-02", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-08-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits nag. even, regular ideas cajole b" }
+{ "l_orderkey": 5763, "l_partkey": 123, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8184.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-23", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "foxes wake slyly. car" }
+{ "l_orderkey": 391, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " escapades sleep furiously about " }
+{ "l_orderkey": 482, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1022.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-20", "l_receiptdate": "1996-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es. quickly ironic escapades sleep furious" }
+{ "l_orderkey": 513, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44973.28, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kages sleep boldly ironic theodolites. acco" }
+{ "l_orderkey": 518, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47017.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ". blithely even ideas cajole furiously. b" }
+{ "l_orderkey": 708, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33729.96, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s boost carefully ruthless theodolites. f" }
+{ "l_orderkey": 801, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al accounts. carefully regular foxes wake" }
+{ "l_orderkey": 868, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43951.16, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly silent deposits wake dar" }
+{ "l_orderkey": 1125, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26575.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l instruction" }
+{ "l_orderkey": 1248, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20442.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-12", "l_commitdate": "1992-03-23", "l_receiptdate": "1992-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal foxes cajole carefully slyl" }
+{ "l_orderkey": 1670, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "fily special ideas " }
 { "l_orderkey": 1890, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 45995.4, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-15", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he carefully regular sauternes. ironic fret" }
 { "l_orderkey": 2016, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8176.96, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "mptotes haggle ideas. packages wake flu" }
 { "l_orderkey": 2177, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 11243.32, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-20", "l_commitdate": "1997-03-07", "l_receiptdate": "1997-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gainst the ca" }
-{ "l_orderkey": 2372, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e carefully blithely even epitaphs. r" }
-{ "l_orderkey": 3491, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " grow against the boldly pending pinto bea" }
-{ "l_orderkey": 4160, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y bold package" }
-{ "l_orderkey": 4706, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23508.76, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deas across t" }
-{ "l_orderkey": 4838, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-23", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly blithely unusual foxes. even package" }
-{ "l_orderkey": 4868, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-26", "l_commitdate": "1997-05-16", "l_receiptdate": "1997-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "osits. final foxes boost regular," }
-{ "l_orderkey": 5666, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-05-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ideas. regular packag" }
-{ "l_orderkey": 5895, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48039.64, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r packages wake carefull" }
-{ "l_orderkey": 513, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44973.28, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-14", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kages sleep boldly ironic theodolites. acco" }
-{ "l_orderkey": 708, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 33729.96, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s boost carefully ruthless theodolites. f" }
-{ "l_orderkey": 868, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43951.16, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kly silent deposits wake dar" }
-{ "l_orderkey": 1248, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20442.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-12", "l_commitdate": "1992-03-23", "l_receiptdate": "1992-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nal foxes cajole carefully slyl" }
-{ "l_orderkey": 1670, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "fily special ideas " }
 { "l_orderkey": 2311, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50083.88, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ideas sleep" }
-{ "l_orderkey": 3395, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39862.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-01-17", "l_receiptdate": "1994-12-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "riously unusual theodolites. fur" }
-{ "l_orderkey": 3585, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ounts use. express, final platelets us" }
-{ "l_orderkey": 3649, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lly bold requests nag; " }
-{ "l_orderkey": 3686, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " furiously unusual accou" }
-{ "l_orderkey": 4039, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17376.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-20", "l_receiptdate": "1998-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " regular foxes haggle carefully bo" }
-{ "l_orderkey": 4161, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic dolphins. in" }
-{ "l_orderkey": 5511, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lphins. carefully blithe de" }
-{ "l_orderkey": 5664, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25553.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-29", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eposits: furiously ironic grouch" }
-{ "l_orderkey": 518, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 47017.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ". blithely even ideas cajole furiously. b" }
-{ "l_orderkey": 801, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10221.2, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al accounts. carefully regular foxes wake" }
+{ "l_orderkey": 2372, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e carefully blithely even epitaphs. r" }
 { "l_orderkey": 2631, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 42929.04, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-12-01", "l_receiptdate": "1994-01-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ect carefully at the furiously final the" }
-{ "l_orderkey": 3842, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-02", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r pinto be" }
-{ "l_orderkey": 4355, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-28", "l_receiptdate": "1997-02-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts affix ironic" }
-{ "l_orderkey": 4387, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " boost slyly ironic instructions. furiou" }
-{ "l_orderkey": 4807, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9199.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-03-01", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "may are blithely. carefully even pinto b" }
-{ "l_orderkey": 391, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " escapades sleep furiously about " }
-{ "l_orderkey": 482, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1022.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-20", "l_receiptdate": "1996-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "es. quickly ironic escapades sleep furious" }
-{ "l_orderkey": 1125, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26575.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-24", "l_commitdate": "1995-01-18", "l_receiptdate": "1995-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l instruction" }
 { "l_orderkey": 2912, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8176.96, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hs cajole over the slyl" }
 { "l_orderkey": 3234, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 51106.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-11", "l_commitdate": "1996-05-19", "l_receiptdate": "1996-06-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly regular ideas according to the regula" }
 { "l_orderkey": 3236, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-23", "l_commitdate": "1996-12-12", "l_receiptdate": "1997-01-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " final pinto " }
+{ "l_orderkey": 3395, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39862.68, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-01-17", "l_receiptdate": "1994-12-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "riously unusual theodolites. fur" }
+{ "l_orderkey": 3491, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-19", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " grow against the boldly pending pinto bea" }
 { "l_orderkey": 3492, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 48039.64, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-03-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deposits. quickly express " }
+{ "l_orderkey": 3585, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-25", "l_receiptdate": "1995-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ounts use. express, final platelets us" }
+{ "l_orderkey": 3649, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lly bold requests nag; " }
+{ "l_orderkey": 3686, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " furiously unusual accou" }
+{ "l_orderkey": 3842, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21464.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-02", "l_receiptdate": "1992-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r pinto be" }
+{ "l_orderkey": 4039, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17376.04, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-01-20", "l_receiptdate": "1998-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " regular foxes haggle carefully bo" }
+{ "l_orderkey": 4160, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y bold package" }
+{ "l_orderkey": 4161, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12265.44, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-25", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "onic dolphins. in" }
+{ "l_orderkey": 4355, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-28", "l_receiptdate": "1997-02-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts affix ironic" }
+{ "l_orderkey": 4387, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3066.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " boost slyly ironic instructions. furiou" }
 { "l_orderkey": 4484, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41906.92, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-25", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ress accounts. ironic deposits unwind fur" }
 { "l_orderkey": 4583, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14309.68, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-08", "l_receiptdate": "1994-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "detect. doggedly regular pi" }
 { "l_orderkey": 4676, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 29641.48, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-29", "l_commitdate": "1995-11-12", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly regular theodolites sleep." }
+{ "l_orderkey": 4706, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23508.76, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deas across t" }
+{ "l_orderkey": 4807, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9199.08, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-03-01", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "may are blithely. carefully even pinto b" }
+{ "l_orderkey": 4838, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35774.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-23", "l_receiptdate": "1992-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly blithely unusual foxes. even package" }
+{ "l_orderkey": 4868, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 22486.64, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-26", "l_commitdate": "1997-05-16", "l_receiptdate": "1997-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "osits. final foxes boost regular," }
 { "l_orderkey": 4869, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30663.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1994-11-20", "l_receiptdate": "1995-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gedly even requests. s" }
 { "l_orderkey": 5350, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19420.28, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-11-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "romise slyly alongsi" }
+{ "l_orderkey": 5511, "l_partkey": 122, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4088.48, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lphins. carefully blithe de" }
+{ "l_orderkey": 5664, "l_partkey": 122, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25553.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-29", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eposits: furiously ironic grouch" }
+{ "l_orderkey": 5666, "l_partkey": 122, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7154.84, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-05-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ideas. regular packag" }
+{ "l_orderkey": 5895, "l_partkey": 122, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 48039.64, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r packages wake carefull" }
 { "l_orderkey": 35, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7147.84, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-01-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the carefully regular " }
 { "l_orderkey": 163, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 13274.56, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inal requests. even pinto beans hag" }
+{ "l_orderkey": 1056, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37781.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " special packages. qui" }
 { "l_orderkey": 1060, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 36760.32, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-04-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r the quickly" }
+{ "l_orderkey": 1377, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39823.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e ironic, regular requests. carefully " }
+{ "l_orderkey": 1410, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15316.8, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-06-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold packages are fluf" }
+{ "l_orderkey": 1764, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y quickly regular packages. car" }
+{ "l_orderkey": 1825, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23485.76, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " wake express, even r" }
 { "l_orderkey": 1890, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 10211.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". even, unusual inst" }
-{ "l_orderkey": 2720, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27570.24, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-08-06", "l_receiptdate": "1993-07-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "eas. carefully regular " }
-{ "l_orderkey": 2919, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50034.88, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1994-02-28", "l_receiptdate": "1993-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "hely final inst" }
-{ "l_orderkey": 4134, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34718.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e furiously regular sheaves sleep" }
-{ "l_orderkey": 4386, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21443.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e pending, sp" }
-{ "l_orderkey": 4903, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1021.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-23", "l_commitdate": "1992-06-13", "l_receiptdate": "1992-05-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nusual requests" }
+{ "l_orderkey": 2053, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ts. fluffily final mul" }
+{ "l_orderkey": 2054, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32675.84, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages thrash. carefully final" }
 { "l_orderkey": 2433, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43908.16, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ular requests. slyly even pa" }
 { "l_orderkey": 2435, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8168.96, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-03", "l_commitdate": "1993-04-02", "l_receiptdate": "1993-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ng the fluffily special foxes nag " }
-{ "l_orderkey": 2563, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5105.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the quickly final theodolite" }
-{ "l_orderkey": 2758, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ptotes sleep furiously" }
-{ "l_orderkey": 2914, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9190.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-17", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. carefully final foxes ar" }
-{ "l_orderkey": 1764, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y quickly regular packages. car" }
 { "l_orderkey": 2469, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30633.6, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-21", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " accounts. regular theodolites affix fu" }
+{ "l_orderkey": 2563, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5105.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the quickly final theodolite" }
+{ "l_orderkey": 2720, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27570.24, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-08-06", "l_receiptdate": "1993-07-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "eas. carefully regular " }
+{ "l_orderkey": 2758, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ptotes sleep furiously" }
+{ "l_orderkey": 2816, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4084.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests print above the final deposits" }
+{ "l_orderkey": 2818, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12253.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lms. quickly bold asymp" }
+{ "l_orderkey": 2914, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 9190.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-17", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. carefully final foxes ar" }
+{ "l_orderkey": 2919, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 50034.88, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1994-02-28", "l_receiptdate": "1993-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "hely final inst" }
 { "l_orderkey": 4004, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23485.76, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " bold theodolites? special packages accordi" }
+{ "l_orderkey": 4134, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34718.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e furiously regular sheaves sleep" }
 { "l_orderkey": 4192, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15316.8, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e slyly special grouches. express pinto b" }
 { "l_orderkey": 4197, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37781.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully enticing decoys boo" }
 { "l_orderkey": 4231, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ublate. theodoli" }
+{ "l_orderkey": 4386, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21443.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-19", "l_receiptdate": "1998-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e pending, sp" }
+{ "l_orderkey": 4903, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1021.12, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-23", "l_commitdate": "1992-06-13", "l_receiptdate": "1992-05-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nusual requests" }
+{ "l_orderkey": 5093, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-11-14", "l_receiptdate": "1994-01-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he final foxes. fluffily ironic " }
+{ "l_orderkey": 5346, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25528.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "he ironic ideas are boldly slyly ironi" }
 { "l_orderkey": 5477, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 19401.28, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-03", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-03-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ost carefully packages." }
 { "l_orderkey": 5538, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4084.48, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-21", "l_commitdate": "1994-02-17", "l_receiptdate": "1994-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ithely along the c" }
 { "l_orderkey": 5763, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 47992.64, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gle slyly. slyly final re" }
 { "l_orderkey": 5858, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20422.4, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uffily unusual pinto beans sleep" }
-{ "l_orderkey": 1056, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37781.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " special packages. qui" }
-{ "l_orderkey": 1377, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39823.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e ironic, regular requests. carefully " }
-{ "l_orderkey": 1410, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15316.8, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-06-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold packages are fluf" }
-{ "l_orderkey": 1825, "l_partkey": 121, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23485.76, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " wake express, even r" }
-{ "l_orderkey": 2053, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-23", "l_commitdate": "1995-03-13", "l_receiptdate": "1995-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ts. fluffily final mul" }
-{ "l_orderkey": 2054, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32675.84, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages thrash. carefully final" }
-{ "l_orderkey": 2816, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4084.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-05", "l_receiptdate": "1994-12-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests print above the final deposits" }
-{ "l_orderkey": 2818, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12253.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-02-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lms. quickly bold asymp" }
-{ "l_orderkey": 5093, "l_partkey": 121, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31654.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-11-14", "l_receiptdate": "1994-01-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he final foxes. fluffily ironic " }
-{ "l_orderkey": 5346, "l_partkey": 121, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25528.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1994-03-19", "l_receiptdate": "1994-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "he ironic ideas are boldly slyly ironi" }
 { "l_orderkey": 35, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 34684.08, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-08", "l_commitdate": "1996-01-15", "l_receiptdate": "1995-11-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". silent, unusual deposits boost" }
+{ "l_orderkey": 36, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 42845.04, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " careful courts. special " }
 { "l_orderkey": 97, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13261.56, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-04-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ayers cajole against the furiously" }
 { "l_orderkey": 132, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43865.16, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-01", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-09-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y pending theodolites" }
 { "l_orderkey": 353, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41824.92, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "refully final theodoli" }
+{ "l_orderkey": 583, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 47945.64, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nts are fluffily. furiously even re" }
 { "l_orderkey": 611, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39784.68, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-10", "l_commitdate": "1993-03-10", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the evenly bold requests. furious" }
 { "l_orderkey": 774, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 2040.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1996-02-10", "l_receiptdate": "1995-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "accounts; slyly regular" }
-{ "l_orderkey": 2720, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 51006.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-07-29", "l_receiptdate": "1993-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l requests. deposits nag furiously" }
-{ "l_orderkey": 2944, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44885.28, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1997-10-28", "l_receiptdate": "1998-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ickly special theodolit" }
-{ "l_orderkey": 3142, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15301.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-15", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "instructions are. ironic packages doz" }
-{ "l_orderkey": 3174, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-07", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic deposits among t" }
-{ "l_orderkey": 3619, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "theodolites detect abo" }
-{ "l_orderkey": 5313, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 21422.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-09-02", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he blithely regular packages. quickly" }
-{ "l_orderkey": 36, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 42845.04, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " careful courts. special " }
 { "l_orderkey": 899, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47945.64, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-05-30", "l_receiptdate": "1998-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "furiously final foxes after the s" }
+{ "l_orderkey": 1221, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41824.92, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-07-02", "l_receiptdate": "1992-05-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ns. bold deposit" }
 { "l_orderkey": 1477, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 33663.96, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-12", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "yly regular p" }
 { "l_orderkey": 1505, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4080.48, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-11-11", "l_receiptdate": "1993-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "side of the s" }
-{ "l_orderkey": 1824, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45905.4, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ent Tiresias. quickly express " }
-{ "l_orderkey": 3175, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28563.36, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-10-05", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ore the even, silent foxes. b" }
-{ "l_orderkey": 3367, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "even packages sleep blithely slyly expr" }
-{ "l_orderkey": 3425, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11221.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckly final deposits use quickly?" }
-{ "l_orderkey": 4292, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35704.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-23", "l_commitdate": "1992-04-04", "l_receiptdate": "1992-04-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dugouts use. furiously bold packag" }
-{ "l_orderkey": 4579, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-15", "l_receiptdate": "1995-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully perman" }
-{ "l_orderkey": 1221, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41824.92, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-07-02", "l_receiptdate": "1992-05-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ns. bold deposit" }
 { "l_orderkey": 1733, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29583.48, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-07-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ns detect among the special accounts. qu" }
+{ "l_orderkey": 1824, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45905.4, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ent Tiresias. quickly express " }
 { "l_orderkey": 1830, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-04-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ely even a" }
 { "l_orderkey": 1959, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15301.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-20", "l_commitdate": "1997-02-18", "l_receiptdate": "1997-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " quickly sp" }
 { "l_orderkey": 2054, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31623.72, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se bold, regular accounts. unusual depos" }
+{ "l_orderkey": 2720, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 51006.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-07-29", "l_receiptdate": "1993-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "l requests. deposits nag furiously" }
+{ "l_orderkey": 2944, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44885.28, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-25", "l_commitdate": "1997-10-28", "l_receiptdate": "1998-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ickly special theodolit" }
 { "l_orderkey": 3013, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35704.2, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-05-04", "l_receiptdate": "1997-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely accord" }
+{ "l_orderkey": 3142, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15301.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-15", "l_commitdate": "1992-08-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "instructions are. ironic packages doz" }
+{ "l_orderkey": 3174, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-07", "l_commitdate": "1996-01-08", "l_receiptdate": "1995-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic deposits among t" }
+{ "l_orderkey": 3175, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28563.36, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-10-05", "l_receiptdate": "1994-10-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ore the even, silent foxes. b" }
+{ "l_orderkey": 3367, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "even packages sleep blithely slyly expr" }
+{ "l_orderkey": 3425, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11221.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckly final deposits use quickly?" }
+{ "l_orderkey": 3619, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38764.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "theodolites detect abo" }
 { "l_orderkey": 4135, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32643.84, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " ideas. requests use. furiously" }
 { "l_orderkey": 4164, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9181.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-08-13", "l_receiptdate": "1998-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "re fluffily slyly bold requests. " }
-{ "l_orderkey": 4903, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27543.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans are; " }
-{ "l_orderkey": 583, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 47945.64, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nts are fluffily. furiously even re" }
+{ "l_orderkey": 4292, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35704.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-23", "l_commitdate": "1992-04-04", "l_receiptdate": "1992-04-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "dugouts use. furiously bold packag" }
+{ "l_orderkey": 4579, "l_partkey": 120, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8160.96, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-15", "l_receiptdate": "1995-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "posits. carefully perman" }
 { "l_orderkey": 4901, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16321.92, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-18", "l_receiptdate": "1998-04-21", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "deposits. blithely fin" }
+{ "l_orderkey": 4903, "l_partkey": 120, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27543.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans are; " }
 { "l_orderkey": 4960, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14281.68, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-04-17", "l_receiptdate": "1995-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "accounts. warhorses are. grouches " }
+{ "l_orderkey": 5313, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 21422.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-09-02", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he blithely regular packages. quickly" }
 { "l_orderkey": 5317, "l_partkey": 120, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 37744.44, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-10-24", "l_receiptdate": "1994-09-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "totes nag theodolites. pend" }
 { "l_orderkey": 5671, "l_partkey": 120, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25503.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-03-28", "l_receiptdate": "1998-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "cording to the quickly final requests-- " }
+{ "l_orderkey": 101, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49936.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-27", "l_receiptdate": "1996-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts-- final packages sleep furiousl" }
+{ "l_orderkey": 256, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40764.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-30", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-12-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nal theodolites. deposits cajole s" }
 { "l_orderkey": 261, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49936.39, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-12", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ions. bold accounts " }
+{ "l_orderkey": 356, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. unusual, final" }
+{ "l_orderkey": 1061, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2038.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-15", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". regular accounts impre" }
+{ "l_orderkey": 1444, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6114.66, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al accounts. br" }
+{ "l_orderkey": 1607, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37707.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-02-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "alongside " }
+{ "l_orderkey": 1857, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8152.88, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "slyly about the fluffily silent req" }
 { "l_orderkey": 1957, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-08-31", "l_receiptdate": "1998-08-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "express packages maintain fluffi" }
 { "l_orderkey": 2276, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5095.55, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-18", "l_receiptdate": "1996-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ias instea" }
 { "l_orderkey": 2279, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 32611.52, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-20", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "re quickly. furiously ironic ide" }
-{ "l_orderkey": 3201, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50955.5, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " deposits. express, ir" }
-{ "l_orderkey": 5959, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ely silent deposits. " }
-{ "l_orderkey": 1444, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6114.66, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-07", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al accounts. br" }
-{ "l_orderkey": 1857, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8152.88, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "slyly about the fluffily silent req" }
 { "l_orderkey": 2306, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21401.31, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-07", "l_commitdate": "1995-09-18", "l_receiptdate": "1995-10-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " ironic pinto " }
-{ "l_orderkey": 3687, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-06-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "foxes cajole quickly about the furiously f" }
-{ "l_orderkey": 5574, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27515.97, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ecial realms. furiously entici" }
-{ "l_orderkey": 101, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49936.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-27", "l_receiptdate": "1996-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts-- final packages sleep furiousl" }
-{ "l_orderkey": 1061, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2038.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-15", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". regular accounts impre" }
-{ "l_orderkey": 1607, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37707.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-02-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "alongside " }
 { "l_orderkey": 2499, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12229.32, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-18", "l_commitdate": "1995-12-13", "l_receiptdate": "1995-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously along the r" }
-{ "l_orderkey": 3590, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve furiously final instructions. slyly regu" }
-{ "l_orderkey": 4675, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1019.11, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-04-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "unts. caref" }
-{ "l_orderkey": 5923, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42802.62, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y regular theodolites w" }
-{ "l_orderkey": 256, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40764.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-30", "l_commitdate": "1993-12-13", "l_receiptdate": "1993-12-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nal theodolites. deposits cajole s" }
-{ "l_orderkey": 356, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s. unusual, final" }
 { "l_orderkey": 2563, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1993-12-31", "l_receiptdate": "1994-02-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lent requests should integrate; carefully e" }
 { "l_orderkey": 2659, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2038.22, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-02-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts above the fluffily express fo" }
+{ "l_orderkey": 3201, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50955.5, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " deposits. express, ir" }
 { "l_orderkey": 3427, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-12", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are carefull" }
+{ "l_orderkey": 3590, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve furiously final instructions. slyly regu" }
+{ "l_orderkey": 3687, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31592.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-06-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "foxes cajole quickly about the furiously f" }
 { "l_orderkey": 3911, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14267.54, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-28", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e blithely brave depo" }
 { "l_orderkey": 4294, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 37707.07, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-13", "l_receiptdate": "1992-09-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cial packages nag f" }
 { "l_orderkey": 4613, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-06-01", "l_receiptdate": "1998-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously express" }
+{ "l_orderkey": 4675, "l_partkey": 119, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1019.11, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-04-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "unts. caref" }
 { "l_orderkey": 4775, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39745.29, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-10-12", "l_receiptdate": "1995-09-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "eep never with the slyly regular acc" }
 { "l_orderkey": 5158, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 20382.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-03", "l_commitdate": "1997-02-20", "l_receiptdate": "1997-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "latelets use accordin" }
 { "l_orderkey": 5219, "l_partkey": 119, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20382.2, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-05-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e along the ironic," }
 { "l_orderkey": 5478, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 25477.75, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-08", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unusual, pending requests haggle accoun" }
-{ "l_orderkey": 1349, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45814.95, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-24", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic, unusual deposits wake carefu" }
-{ "l_orderkey": 1793, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests nod ac" }
-{ "l_orderkey": 2308, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24434.64, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1992-12-24", "l_receiptdate": "1993-03-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts sleep. busy excuses along the s" }
-{ "l_orderkey": 2725, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23416.53, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular deposits. brave foxes " }
-{ "l_orderkey": 2948, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-09-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unusual excuses use about the " }
-{ "l_orderkey": 5188, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p according to the sometimes regu" }
-{ "l_orderkey": 5766, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-10-30", "l_receiptdate": "1993-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly even requests. furiou" }
-{ "l_orderkey": 226, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ep carefully regular accounts. ironic" }
-{ "l_orderkey": 902, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8144.88, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " orbits al" }
-{ "l_orderkey": 934, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y unusual requests dazzle above t" }
-{ "l_orderkey": 1441, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "special requests ha" }
-{ "l_orderkey": 1475, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al deposits use. ironic packages along the " }
-{ "l_orderkey": 1734, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-07-17", "l_receiptdate": "1994-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "final warhorses." }
-{ "l_orderkey": 2629, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6108.66, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-05-29", "l_receiptdate": "1998-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dolites hinder bli" }
-{ "l_orderkey": 3201, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27488.97, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-08-24", "l_receiptdate": "1993-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits are slyly along" }
-{ "l_orderkey": 4035, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1018.11, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. quickly " }
-{ "l_orderkey": 4386, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28507.08, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-03-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". quick packages play slyly " }
-{ "l_orderkey": 5314, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10181.1, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "latelets haggle final" }
-{ "l_orderkey": 5857, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12217.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-02-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts. express, final" }
+{ "l_orderkey": 5574, "l_partkey": 119, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27515.97, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ecial realms. furiously entici" }
+{ "l_orderkey": 5923, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42802.62, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y regular theodolites w" }
+{ "l_orderkey": 5959, "l_partkey": 119, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 35668.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ely silent deposits. " }
 { "l_orderkey": 133, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29525.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-28", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-03-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " the carefully regular theodoli" }
 { "l_orderkey": 226, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 2036.22, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "al platelets. express somas " }
+{ "l_orderkey": 226, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ep carefully regular accounts. ironic" }
+{ "l_orderkey": 293, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13235.43, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-17", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " wake after the quickly even deposits. bli" }
 { "l_orderkey": 454, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24434.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-26", "l_commitdate": "1996-03-23", "l_receiptdate": "1996-05-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "le. deposits after the ideas nag unusual pa" }
+{ "l_orderkey": 514, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34615.74, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ily even patterns. bold, silent instruc" }
 { "l_orderkey": 610, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26470.86, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cross the furiously even theodolites sl" }
+{ "l_orderkey": 902, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8144.88, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " orbits al" }
+{ "l_orderkey": 934, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y unusual requests dazzle above t" }
 { "l_orderkey": 961, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 7126.77, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-23", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "usual dolphins. ironic pearls sleep blit" }
 { "l_orderkey": 999, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15271.65, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-12", "l_commitdate": "1993-10-18", "l_receiptdate": "1994-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y ironic requests. carefully regu" }
+{ "l_orderkey": 1349, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45814.95, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-24", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic, unusual deposits wake carefu" }
+{ "l_orderkey": 1379, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50905.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "olphins. ca" }
+{ "l_orderkey": 1441, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 14253.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "special requests ha" }
+{ "l_orderkey": 1475, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al deposits use. ironic packages along the " }
+{ "l_orderkey": 1664, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " use. ironic deposits integrate. slyly unu" }
+{ "l_orderkey": 1734, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-20", "l_commitdate": "1994-07-17", "l_receiptdate": "1994-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "final warhorses." }
+{ "l_orderkey": 1793, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "equests nod ac" }
 { "l_orderkey": 2213, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20362.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-21", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "iously express accounts; " }
+{ "l_orderkey": 2308, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24434.64, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1992-12-24", "l_receiptdate": "1993-03-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts sleep. busy excuses along the s" }
 { "l_orderkey": 2374, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41742.51, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "heodolites. requests" }
+{ "l_orderkey": 2629, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6108.66, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-05-29", "l_receiptdate": "1998-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dolites hinder bli" }
 { "l_orderkey": 2631, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15271.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-30", "l_commitdate": "1993-11-06", "l_receiptdate": "1993-10-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y. furiously even pinto be" }
+{ "l_orderkey": 2725, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23416.53, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular deposits. brave foxes " }
+{ "l_orderkey": 2756, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits grow bold sheaves; iro" }
+{ "l_orderkey": 2948, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-29", "l_commitdate": "1994-10-23", "l_receiptdate": "1994-09-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unusual excuses use about the " }
+{ "l_orderkey": 3201, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27488.97, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-08-24", "l_receiptdate": "1993-09-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits are slyly along" }
 { "l_orderkey": 3236, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7126.77, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-27", "l_commitdate": "1996-12-18", "l_receiptdate": "1997-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dolites. slyly unus" }
 { "l_orderkey": 3428, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-01", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly regular pinto beans sleep" }
 { "l_orderkey": 3747, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23416.53, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-11-10", "l_receiptdate": "1996-12-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages are ironic" }
+{ "l_orderkey": 4035, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1018.11, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " requests. quickly " }
+{ "l_orderkey": 4386, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28507.08, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-03-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". quick packages play slyly " }
+{ "l_orderkey": 5188, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18325.98, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-19", "l_receiptdate": "1995-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "p according to the sometimes regu" }
 { "l_orderkey": 5282, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36651.96, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-20", "l_commitdate": "1998-04-10", "l_receiptdate": "1998-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "re slyly accor" }
+{ "l_orderkey": 5314, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10181.1, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "latelets haggle final" }
 { "l_orderkey": 5408, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-10-17", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "thely ironic requests alongside of the sl" }
-{ "l_orderkey": 293, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13235.43, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-17", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " wake after the quickly even deposits. bli" }
-{ "l_orderkey": 514, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34615.74, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ily even patterns. bold, silent instruc" }
-{ "l_orderkey": 1379, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50905.5, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "olphins. ca" }
-{ "l_orderkey": 1664, "l_partkey": 118, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48869.28, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " use. ironic deposits integrate. slyly unu" }
-{ "l_orderkey": 2756, "l_partkey": 118, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35633.85, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits grow bold sheaves; iro" }
+{ "l_orderkey": 5766, "l_partkey": 118, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4072.44, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1993-10-30", "l_receiptdate": "1993-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly even requests. furiou" }
+{ "l_orderkey": 5857, "l_partkey": 118, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12217.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1997-12-27", "l_receiptdate": "1998-02-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "counts. express, final" }
 { "l_orderkey": 288, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49838.39, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ic excuses sleep always spe" }
+{ "l_orderkey": 353, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ual accounts! carefu" }
+{ "l_orderkey": 705, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35598.85, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "carefully ironic accounts" }
 { "l_orderkey": 807, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49838.39, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-13", "l_receiptdate": "1993-12-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " furiously according to the un" }
+{ "l_orderkey": 960, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-01", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts. fluffily regular requests " }
+{ "l_orderkey": 966, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42718.62, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tions boost furiously car" }
 { "l_orderkey": 1508, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 38650.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-30", "l_commitdate": "1998-06-23", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tes wake furiously regular w" }
 { "l_orderkey": 1575, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 48821.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-10-25", "l_receiptdate": "1995-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "cies. regu" }
-{ "l_orderkey": 2790, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50855.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fter the regular ideas. f" }
-{ "l_orderkey": 3079, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38650.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-25", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully regular realms" }
-{ "l_orderkey": 3270, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual packages" }
-{ "l_orderkey": 4833, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11188.21, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s nag above the busily sile" }
-{ "l_orderkey": 353, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ual accounts! carefu" }
-{ "l_orderkey": 966, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 42718.62, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tions boost furiously car" }
 { "l_orderkey": 1794, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23393.53, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-11-30", "l_receiptdate": "1997-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "usly unusual theodolites doze about " }
 { "l_orderkey": 1794, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 47804.17, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1997-11-30", "l_receiptdate": "1998-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " haggle slyly. furiously express orbit" }
+{ "l_orderkey": 1856, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20342.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-05-06", "l_receiptdate": "1992-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ost carefully. slyly bold accounts" }
 { "l_orderkey": 2339, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13222.43, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-10", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ges. blithely special depend" }
 { "l_orderkey": 2342, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24410.64, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-07-22", "l_receiptdate": "1996-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nstructions c" }
 { "l_orderkey": 2436, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18307.98, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y ironic accounts. furiously even packa" }
+{ "l_orderkey": 2790, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50855.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-11-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fter the regular ideas. f" }
+{ "l_orderkey": 3079, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38650.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-11-25", "l_receiptdate": "1997-12-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e carefully regular realms" }
+{ "l_orderkey": 3236, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10171.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "arefully. fluffily reg" }
+{ "l_orderkey": 3270, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9153.99, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-14", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual packages" }
+{ "l_orderkey": 3360, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29496.19, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "hely gifts. spe" }
 { "l_orderkey": 3393, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 16273.76, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-08-19", "l_receiptdate": "1995-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uses. instructions after the blithely " }
 { "l_orderkey": 3494, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40684.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lites haggle furiously about the fin" }
+{ "l_orderkey": 3526, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23393.53, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "special, regular packages cajole. " }
 { "l_orderkey": 3617, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46787.06, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-06-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar theodolites. regu" }
 { "l_orderkey": 3623, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33564.63, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-13", "l_receiptdate": "1997-04-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "odolites. blithely spe" }
+{ "l_orderkey": 3648, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s requests. silent asymp" }
 { "l_orderkey": 3686, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7119.77, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-16", "l_commitdate": "1998-09-02", "l_receiptdate": "1998-07-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ake carefully carefully q" }
 { "l_orderkey": 3782, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34581.74, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gage after the even" }
 { "l_orderkey": 4193, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 3051.33, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-29", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "osits above the depo" }
+{ "l_orderkey": 4545, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27461.97, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-02-22", "l_receiptdate": "1993-03-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ccounts haggle carefully. deposits " }
 { "l_orderkey": 4678, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18307.98, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-22", "l_receiptdate": "1998-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usly ironic " }
-{ "l_orderkey": 705, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 35598.85, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "carefully ironic accounts" }
-{ "l_orderkey": 3236, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10171.1, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "arefully. fluffily reg" }
-{ "l_orderkey": 3360, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29496.19, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "hely gifts. spe" }
-{ "l_orderkey": 3526, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23393.53, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-01", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-05-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "special, regular packages cajole. " }
-{ "l_orderkey": 3648, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s requests. silent asymp" }
+{ "l_orderkey": 4833, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11188.21, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s nag above the busily sile" }
 { "l_orderkey": 5156, "l_partkey": 117, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21359.31, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1997-01-30", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts detect against the furiously reg" }
 { "l_orderkey": 5410, "l_partkey": 117, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48821.28, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-09-11", "l_receiptdate": "1998-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " about the slyly even courts. quickly regul" }
-{ "l_orderkey": 5920, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2034.22, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " evenly spe" }
-{ "l_orderkey": 960, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25427.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-01", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts. fluffily regular requests " }
-{ "l_orderkey": 1856, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20342.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-05-06", "l_receiptdate": "1992-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ost carefully. slyly bold accounts" }
-{ "l_orderkey": 4545, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 27461.97, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-02-22", "l_receiptdate": "1993-03-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ccounts haggle carefully. deposits " }
 { "l_orderkey": 5508, "l_partkey": 117, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4068.44, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-09-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fluffily about the even " }
+{ "l_orderkey": 5920, "l_partkey": 117, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2034.22, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-03-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " evenly spe" }
 { "l_orderkey": 66, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31499.41, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ut the unusual accounts sleep at the bo" }
 { "l_orderkey": 69, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48773.28, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-09-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "regular epitaphs. carefully even ideas hag" }
 { "l_orderkey": 100, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22354.42, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nto beans alongside of the fi" }
+{ "l_orderkey": 130, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13209.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending dolphins sleep furious" }
+{ "l_orderkey": 514, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43692.73, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular " }
+{ "l_orderkey": 1861, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 38612.18, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pending deposits cajole quic" }
 { "l_orderkey": 1925, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e carefully regul" }
+{ "l_orderkey": 2148, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21338.31, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-28", "l_commitdate": "1995-05-26", "l_receiptdate": "1995-06-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "deposits ag" }
+{ "l_orderkey": 2241, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 22354.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", ironic depen" }
+{ "l_orderkey": 2437, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular deposits. ironic fray" }
+{ "l_orderkey": 2534, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-29", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sual depos" }
+{ "l_orderkey": 2755, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48773.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-10", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "yly even epitaphs for the " }
 { "l_orderkey": 3136, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26418.86, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-11-07", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eep fluffily. daringly silent attainments d" }
+{ "l_orderkey": 3200, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17273.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-06", "l_commitdate": "1996-04-21", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "side of the furiously pendin" }
+{ "l_orderkey": 3619, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27434.97, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pecial accounts haggle care" }
+{ "l_orderkey": 3682, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18289.98, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-05-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "regular dependencies" }
+{ "l_orderkey": 3905, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ully furiously furious packag" }
 { "l_orderkey": 4007, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41660.51, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eposits. regular epitaphs boost blithely." }
 { "l_orderkey": 4421, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 18289.98, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". regular, s" }
+{ "l_orderkey": 4547, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "slyly express a" }
+{ "l_orderkey": 4706, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5080.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ptotes haggle ca" }
 { "l_orderkey": 4711, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 45724.95, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-07-14", "l_receiptdate": "1998-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic theodolites " }
 { "l_orderkey": 4900, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "heodolites. request" }
-{ "l_orderkey": 514, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 43692.73, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular " }
-{ "l_orderkey": 2241, "l_partkey": 116, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 22354.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-13", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-08-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", ironic depen" }
-{ "l_orderkey": 3200, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17273.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-06", "l_commitdate": "1996-04-21", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "side of the furiously pendin" }
-{ "l_orderkey": 3682, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18289.98, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-05-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "regular dependencies" }
-{ "l_orderkey": 4706, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 5080.55, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ptotes haggle ca" }
-{ "l_orderkey": 5603, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49789.39, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully silent requests. carefully fin" }
-{ "l_orderkey": 2148, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21338.31, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-28", "l_commitdate": "1995-05-26", "l_receiptdate": "1995-06-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "deposits ag" }
-{ "l_orderkey": 2534, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-29", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sual depos" }
-{ "l_orderkey": 3619, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27434.97, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "pecial accounts haggle care" }
-{ "l_orderkey": 3905, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ully furiously furious packag" }
-{ "l_orderkey": 4547, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7112.77, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "slyly express a" }
-{ "l_orderkey": 130, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13209.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-07-29", "l_receiptdate": "1992-07-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending dolphins sleep furious" }
-{ "l_orderkey": 1861, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 38612.18, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pending deposits cajole quic" }
-{ "l_orderkey": 2437, "l_partkey": 116, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12193.32, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely regular deposits. ironic fray" }
-{ "l_orderkey": 2755, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48773.28, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-10", "l_receiptdate": "1992-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "yly even epitaphs for the " }
 { "l_orderkey": 4901, "l_partkey": 116, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40644.4, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-08", "l_commitdate": "1998-01-30", "l_receiptdate": "1998-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ect across the furiou" }
-{ "l_orderkey": 646, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic packages sleep across th" }
-{ "l_orderkey": 1188, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2030.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its breach blit" }
-{ "l_orderkey": 1543, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6090.66, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " among the carefully bold or" }
-{ "l_orderkey": 1606, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21317.31, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-02", "l_receiptdate": "1997-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending theodolites prom" }
-{ "l_orderkey": 3937, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27407.97, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-06", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-02-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven ideas. slyly expr" }
-{ "l_orderkey": 5440, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3045.33, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y. accounts haggle along the blit" }
-{ "l_orderkey": 5794, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14211.54, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uriously carefully ironic reque" }
-{ "l_orderkey": 5921, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5075.55, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eas cajole across the final, fi" }
+{ "l_orderkey": 5603, "l_partkey": 116, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49789.39, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully silent requests. carefully fin" }
 { "l_orderkey": 132, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32483.52, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "d instructions hagg" }
-{ "l_orderkey": 900, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cial pinto beans nag " }
-{ "l_orderkey": 1094, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9135.99, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as. slyly pe" }
-{ "l_orderkey": 1504, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final theodolites. furiously e" }
-{ "l_orderkey": 2880, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42634.62, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ions. carefully final accounts are unusual," }
-{ "l_orderkey": 3365, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13196.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-01-31", "l_receiptdate": "1995-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pths wake r" }
-{ "l_orderkey": 3973, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37559.07, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "inos wake fluffily. pending requests nag " }
-{ "l_orderkey": 4101, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22332.42, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly express instructions. careful" }
-{ "l_orderkey": 4644, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-02-28", "l_receiptdate": "1998-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits according to the" }
-{ "l_orderkey": 4930, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20302.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he carefully" }
-{ "l_orderkey": 5191, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests! ironic theodolites cajole care" }
-{ "l_orderkey": 5317, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ts about the packages cajole furio" }
-{ "l_orderkey": 1477, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-31", "l_receiptdate": "1998-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. final pearls kindle. accounts " }
-{ "l_orderkey": 1829, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 36543.96, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ackages-- express requests sleep; pen" }
-{ "l_orderkey": 2912, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18271.98, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-13", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts cajole reg" }
-{ "l_orderkey": 3077, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23347.53, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly. fluffily pending dinos across" }
-{ "l_orderkey": 3362, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "packages haggle furi" }
-{ "l_orderkey": 5093, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30453.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ithely ironic sheaves use fluff" }
-{ "l_orderkey": 5927, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8120.88, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-24", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ilent dependencies nod c" }
 { "l_orderkey": 135, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13196.43, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-12-22", "l_receiptdate": "1995-11-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nal ideas. final instr" }
 { "l_orderkey": 452, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2030.22, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y express instru" }
+{ "l_orderkey": 646, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic packages sleep across th" }
+{ "l_orderkey": 900, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-22", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cial pinto beans nag " }
+{ "l_orderkey": 1094, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9135.99, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as. slyly pe" }
+{ "l_orderkey": 1188, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2030.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-22", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its breach blit" }
+{ "l_orderkey": 1477, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-10-31", "l_receiptdate": "1998-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. final pearls kindle. accounts " }
+{ "l_orderkey": 1504, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-10-22", "l_receiptdate": "1992-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final theodolites. furiously e" }
+{ "l_orderkey": 1543, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6090.66, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " among the carefully bold or" }
+{ "l_orderkey": 1606, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21317.31, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-02", "l_receiptdate": "1997-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending theodolites prom" }
+{ "l_orderkey": 1829, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 36543.96, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ackages-- express requests sleep; pen" }
 { "l_orderkey": 1831, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17256.87, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s boost ironic foxe" }
 { "l_orderkey": 2084, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 15226.65, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tithes. bravely pendi" }
 { "l_orderkey": 2565, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34513.74, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nstructions was carefu" }
 { "l_orderkey": 2657, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22332.42, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "r ideas. furiously special dolphins" }
+{ "l_orderkey": 2880, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42634.62, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ions. carefully final accounts are unusual," }
+{ "l_orderkey": 2912, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18271.98, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-13", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts cajole reg" }
+{ "l_orderkey": 3077, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23347.53, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-05", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly. fluffily pending dinos across" }
+{ "l_orderkey": 3362, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40604.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "packages haggle furi" }
+{ "l_orderkey": 3365, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13196.43, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-01-31", "l_receiptdate": "1995-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "pths wake r" }
+{ "l_orderkey": 3937, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27407.97, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-06", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-02-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven ideas. slyly expr" }
+{ "l_orderkey": 3973, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37559.07, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-29", "l_commitdate": "1992-05-04", "l_receiptdate": "1992-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "inos wake fluffily. pending requests nag " }
+{ "l_orderkey": 4101, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22332.42, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly express instructions. careful" }
+{ "l_orderkey": 4644, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10151.1, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-02-28", "l_receiptdate": "1998-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits according to the" }
+{ "l_orderkey": 4930, "l_partkey": 115, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20302.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-21", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-08-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "he carefully" }
+{ "l_orderkey": 5093, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 30453.3, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-11-27", "l_receiptdate": "1993-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ithely ironic sheaves use fluff" }
+{ "l_orderkey": 5191, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41619.51, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests! ironic theodolites cajole care" }
+{ "l_orderkey": 5317, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48725.28, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-25", "l_receiptdate": "1994-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ts about the packages cajole furio" }
+{ "l_orderkey": 5440, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3045.33, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y. accounts haggle along the blit" }
+{ "l_orderkey": 5794, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14211.54, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "uriously carefully ironic reque" }
 { "l_orderkey": 5798, "l_partkey": 115, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 32483.52, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-05-03", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ubt blithely above the " }
+{ "l_orderkey": 5921, "l_partkey": 115, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 5075.55, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-01", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eas cajole across the final, fi" }
+{ "l_orderkey": 5927, "l_partkey": 115, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8120.88, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-24", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ilent dependencies nod c" }
+{ "l_orderkey": 355, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31437.41, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y unusual, ironic" }
 { "l_orderkey": 357, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 26366.86, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-11-26", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " carefully pending accounts use a" }
 { "l_orderkey": 710, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es. furiously p" }
+{ "l_orderkey": 1253, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19268.09, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-04-22", "l_receiptdate": "1993-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al pinto bea" }
+{ "l_orderkey": 1318, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24338.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ual, unusual packages. fluffy, iro" }
+{ "l_orderkey": 1440, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 46649.06, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely even instructions. " }
 { "l_orderkey": 1506, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30423.3, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-22", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "deposits cajole " }
 { "l_orderkey": 1604, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 19268.09, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " ideas. bol" }
+{ "l_orderkey": 1635, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20282.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "oost according to the carefully even accou" }
 { "l_orderkey": 1636, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 7098.77, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-09-10", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ronic instructions. final" }
 { "l_orderkey": 1795, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34479.74, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "closely regular instructions wake. " }
 { "l_orderkey": 2469, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16225.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-19", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-03-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ing asymptotes " }
 { "l_orderkey": 2532, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9126.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-30", "l_commitdate": "1995-11-23", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "cial ideas haggle slyly pending request" }
-{ "l_orderkey": 4004, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39550.29, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts sleep furious" }
-{ "l_orderkey": 5090, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2028.22, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tes. slowly iro" }
-{ "l_orderkey": 355, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31437.41, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y unusual, ironic" }
-{ "l_orderkey": 1440, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 46649.06, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-21", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely even instructions. " }
-{ "l_orderkey": 1635, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20282.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "oost according to the carefully even accou" }
 { "l_orderkey": 2663, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35493.85, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1995-10-16", "l_receiptdate": "1996-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tect. slyly fina" }
 { "l_orderkey": 2692, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-11", "l_commitdate": "1998-02-11", "l_receiptdate": "1998-03-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "posits. final, express requests nag furi" }
-{ "l_orderkey": 4196, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42592.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " instructions. courts cajole slyly ev" }
-{ "l_orderkey": 1253, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19268.09, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-04-22", "l_receiptdate": "1993-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "al pinto bea" }
-{ "l_orderkey": 1318, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24338.64, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ual, unusual packages. fluffy, iro" }
-{ "l_orderkey": 4006, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 25352.75, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-02-09", "l_receiptdate": "1995-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests use depos" }
-{ "l_orderkey": 5281, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37522.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ronic dependencies. fluffily final p" }
-{ "l_orderkey": 5671, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30423.3, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily ironi" }
 { "l_orderkey": 3014, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 14197.54, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-19", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". slyly brave platelets nag. careful," }
+{ "l_orderkey": 4004, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39550.29, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts sleep furious" }
+{ "l_orderkey": 4006, "l_partkey": 114, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 25352.75, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-02-09", "l_receiptdate": "1995-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests use depos" }
+{ "l_orderkey": 4196, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42592.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " instructions. courts cajole slyly ev" }
 { "l_orderkey": 4452, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21296.31, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "multipliers x-ray carefully in place of " }
+{ "l_orderkey": 5090, "l_partkey": 114, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2028.22, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-04-23", "l_receiptdate": "1997-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "tes. slowly iro" }
+{ "l_orderkey": 5281, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37522.07, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1995-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ronic dependencies. fluffily final p" }
 { "l_orderkey": 5345, "l_partkey": 114, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 37522.07, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-01", "l_commitdate": "1997-10-09", "l_receiptdate": "1997-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " along the ironically fina" }
+{ "l_orderkey": 5671, "l_partkey": 114, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30423.3, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily ironi" }
+{ "l_orderkey": 647, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5065.55, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express packages haggle caref" }
+{ "l_orderkey": 964, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1013.11, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-20", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts. quickly even platelets s" }
 { "l_orderkey": 1027, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20262.2, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-08-29", "l_receiptdate": "1992-06-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar excuses eat f" }
 { "l_orderkey": 1090, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28367.08, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-20", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s cajole above the regular" }
 { "l_orderkey": 1188, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9117.99, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-04", "l_commitdate": "1996-06-04", "l_receiptdate": "1996-08-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ow carefully ironic d" }
-{ "l_orderkey": 2050, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41537.51, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-08-27", "l_receiptdate": "1994-06-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " final theodolites. depende" }
-{ "l_orderkey": 3328, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6078.66, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-01-25", "l_receiptdate": "1993-03-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ffily even instructions detect b" }
-{ "l_orderkey": 3363, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20262.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully quiet excuses wake. sl" }
-{ "l_orderkey": 3875, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-13", "l_receiptdate": "1997-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sleep furiously about the deposits. quickl" }
-{ "l_orderkey": 4741, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16209.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "final foxes haggle r" }
-{ "l_orderkey": 5411, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-04", "l_receiptdate": "1997-07-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding, special foxes unw" }
-{ "l_orderkey": 647, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5065.55, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express packages haggle caref" }
-{ "l_orderkey": 964, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1013.11, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-20", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "unts. quickly even platelets s" }
 { "l_orderkey": 1347, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28367.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-08-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes after the blithely special i" }
-{ "l_orderkey": 2054, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11144.21, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accou" }
-{ "l_orderkey": 2214, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42550.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ons. deposi" }
-{ "l_orderkey": 2759, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37485.07, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-22", "l_receiptdate": "1994-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lar Tiresias affix ironically carefully sp" }
-{ "l_orderkey": 4487, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sual packages should ha" }
-{ "l_orderkey": 5862, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4052.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent deposit" }
-{ "l_orderkey": 5890, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38498.18, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1992-12-09", "l_receiptdate": "1993-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " accounts. carefully final asymptotes" }
-{ "l_orderkey": 3207, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2026.22, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "among the ironic, even packages " }
-{ "l_orderkey": 3333, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "riously ironic r" }
-{ "l_orderkey": 3911, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ss theodolites are blithely along t" }
-{ "l_orderkey": 4263, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47616.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-07-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y. theodolites wake idly ironic do" }
 { "l_orderkey": 1860, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9117.99, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-03", "l_commitdate": "1996-05-31", "l_receiptdate": "1996-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "c realms print carefully car" }
 { "l_orderkey": 1892, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48629.28, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-16", "l_commitdate": "1994-06-16", "l_receiptdate": "1994-06-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tornis detect regul" }
+{ "l_orderkey": 2050, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41537.51, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-08-27", "l_receiptdate": "1994-06-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " final theodolites. depende" }
+{ "l_orderkey": 2054, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11144.21, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accou" }
 { "l_orderkey": 2081, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 19249.09, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-01", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s affix sometimes express requests. quickly" }
+{ "l_orderkey": 2214, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42550.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ons. deposi" }
+{ "l_orderkey": 2759, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37485.07, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-05", "l_commitdate": "1994-02-22", "l_receiptdate": "1994-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lar Tiresias affix ironically carefully sp" }
+{ "l_orderkey": 3207, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2026.22, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "among the ironic, even packages " }
+{ "l_orderkey": 3328, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6078.66, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-01-25", "l_receiptdate": "1993-03-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ffily even instructions detect b" }
+{ "l_orderkey": 3333, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "riously ironic r" }
+{ "l_orderkey": 3363, "l_partkey": 113, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 20262.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-11", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully quiet excuses wake. sl" }
 { "l_orderkey": 3651, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41537.51, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-10", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-05-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "blithely. furiously " }
 { "l_orderkey": 3750, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47616.17, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-11", "l_commitdate": "1995-06-13", "l_receiptdate": "1995-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "slowly regular accounts. blithely ev" }
+{ "l_orderkey": 3875, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-13", "l_receiptdate": "1997-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sleep furiously about the deposits. quickl" }
+{ "l_orderkey": 3911, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ss theodolites are blithely along t" }
 { "l_orderkey": 4160, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25327.75, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-09-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar accounts sleep blithe" }
+{ "l_orderkey": 4263, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47616.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-07-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y. theodolites wake idly ironic do" }
+{ "l_orderkey": 4487, "l_partkey": 113, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 49642.39, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sual packages should ha" }
+{ "l_orderkey": 4741, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16209.76, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-25", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "final foxes haggle r" }
+{ "l_orderkey": 5411, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10131.1, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-04", "l_receiptdate": "1997-07-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nding, special foxes unw" }
+{ "l_orderkey": 5862, "l_partkey": 113, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4052.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-26", "l_receiptdate": "1997-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly silent deposit" }
+{ "l_orderkey": 5890, "l_partkey": 113, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38498.18, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-14", "l_commitdate": "1992-12-09", "l_receiptdate": "1993-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " accounts. carefully final asymptotes" }
 { "l_orderkey": 289, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 6072.66, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "d packages use fluffily furiously" }
-{ "l_orderkey": 1028, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39472.29, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " final dependencies affix a" }
-{ "l_orderkey": 1095, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " bold accounts haggle slyly furiously even" }
-{ "l_orderkey": 2113, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1998-01-08", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly regular accounts hinder about the" }
-{ "l_orderkey": 2305, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully alongside of " }
-{ "l_orderkey": 4769, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15181.65, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular platelets can cajole across the " }
-{ "l_orderkey": 5827, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38460.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-18", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly ruthless accounts" }
 { "l_orderkey": 481, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31375.41, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-15", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly final packages believe. quick" }
 { "l_orderkey": 768, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43520.73, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sual ideas wake quickly" }
 { "l_orderkey": 833, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38460.18, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-04-21", "l_receiptdate": "1994-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " platelets promise furiously. " }
-{ "l_orderkey": 2564, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y express requests sleep furi" }
-{ "l_orderkey": 2885, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pending packages wake. " }
-{ "l_orderkey": 2887, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fily final packages. regula" }
-{ "l_orderkey": 3585, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11133.21, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "even packages" }
-{ "l_orderkey": 5313, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 47569.17, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-12", "l_commitdate": "1997-08-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "pinto beans across the " }
-{ "l_orderkey": 5664, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44532.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-09-26", "l_receiptdate": "1998-10-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ang thinly bold pa" }
-{ "l_orderkey": 5888, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing to the spe" }
-{ "l_orderkey": 1698, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19230.09, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-06-21", "l_receiptdate": "1997-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " fluffily e" }
-{ "l_orderkey": 3713, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41496.51, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-07-17", "l_receiptdate": "1998-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits wake blithely fina" }
-{ "l_orderkey": 4071, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22266.42, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sits cajole carefully final instructio" }
-{ "l_orderkey": 5955, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "oss the fluffily regular" }
+{ "l_orderkey": 1028, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39472.29, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-18", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " final dependencies affix a" }
+{ "l_orderkey": 1095, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " bold accounts haggle slyly furiously even" }
 { "l_orderkey": 1475, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 23278.53, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-13", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-03-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "hely regular hocke" }
+{ "l_orderkey": 1698, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19230.09, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-06-21", "l_receiptdate": "1997-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " fluffily e" }
+{ "l_orderkey": 2113, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1998-01-08", "l_receiptdate": "1998-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly regular accounts hinder about the" }
+{ "l_orderkey": 2305, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully alongside of " }
+{ "l_orderkey": 2564, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y express requests sleep furi" }
 { "l_orderkey": 2567, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50605.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully pending foxes are furi" }
 { "l_orderkey": 2759, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11133.21, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-02-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "hely regular " }
+{ "l_orderkey": 2885, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4048.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " pending packages wake. " }
+{ "l_orderkey": 2887, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17205.87, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-31", "l_commitdate": "1997-07-04", "l_receiptdate": "1997-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "fily final packages. regula" }
 { "l_orderkey": 2982, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 21254.31, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-06-08", "l_receiptdate": "1995-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ironic deposits. furiously ex" }
+{ "l_orderkey": 3585, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11133.21, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-14", "l_receiptdate": "1995-01-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "even packages" }
+{ "l_orderkey": 3713, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41496.51, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-11", "l_commitdate": "1998-07-17", "l_receiptdate": "1998-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eposits wake blithely fina" }
 { "l_orderkey": 3907, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41496.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-13", "l_commitdate": "1992-10-23", "l_receiptdate": "1992-09-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ackages wake along the carefully regul" }
+{ "l_orderkey": 4071, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22266.42, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-12-14", "l_receiptdate": "1996-11-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sits cajole carefully final instructio" }
+{ "l_orderkey": 4769, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15181.65, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "egular platelets can cajole across the " }
 { "l_orderkey": 5024, "l_partkey": 112, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 18217.98, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-02", "l_commitdate": "1997-01-16", "l_receiptdate": "1996-12-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "zle carefully sauternes. quickly" }
 { "l_orderkey": 5153, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36435.96, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-15", "l_commitdate": "1995-11-08", "l_receiptdate": "1995-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ironic instru" }
+{ "l_orderkey": 5313, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 47569.17, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-12", "l_commitdate": "1997-08-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "pinto beans across the " }
+{ "l_orderkey": 5664, "l_partkey": 112, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44532.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-09-26", "l_receiptdate": "1998-10-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ang thinly bold pa" }
+{ "l_orderkey": 5827, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 38460.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-18", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly ruthless accounts" }
+{ "l_orderkey": 5888, "l_partkey": 112, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24290.64, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ing to the spe" }
+{ "l_orderkey": 5955, "l_partkey": 112, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 40484.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-06-11", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "oss the fluffily regular" }
 { "l_orderkey": 192, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 15166.65, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-30", "l_commitdate": "1998-02-10", "l_receiptdate": "1998-02-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "he ironic requests haggle about" }
-{ "l_orderkey": 902, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-01", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "into beans thrash blithely about the flu" }
-{ "l_orderkey": 1252, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27299.97, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-22", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "packages hag" }
-{ "l_orderkey": 2180, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 47522.17, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending, regular ideas. iron" }
-{ "l_orderkey": 3456, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34377.74, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-29", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usy pinto beans b" }
-{ "l_orderkey": 4613, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e blithely against the even, bold pi" }
 { "l_orderkey": 416, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22244.42, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1993-12-17", "l_receiptdate": "1994-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "rint blithely above the pending sentim" }
-{ "l_orderkey": 2274, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23255.53, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-11-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kly special warhorse" }
-{ "l_orderkey": 4002, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eep. quickly" }
-{ "l_orderkey": 4385, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38422.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inal frays. final, bold exc" }
-{ "l_orderkey": 4674, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " regular requests na" }
-{ "l_orderkey": 5121, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45499.95, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-09-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial accounts cajole ca" }
-{ "l_orderkey": 5254, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntegrate carefully among the pending" }
 { "l_orderkey": 610, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49544.39, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-29", "l_commitdate": "1995-10-26", "l_receiptdate": "1995-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ular instruc" }
-{ "l_orderkey": 1575, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39433.29, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-11-05", "l_receiptdate": "1995-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " after the unusual asym" }
-{ "l_orderkey": 3206, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37411.07, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-10-31", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quick theodolites hagg" }
-{ "l_orderkey": 4705, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22244.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " fluffily pending accounts ca" }
-{ "l_orderkey": 5413, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 36399.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-12", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular, regular ideas mold! final requests" }
-{ "l_orderkey": 5731, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6066.66, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-02", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits integrate slyly close platelets. quick" }
+{ "l_orderkey": 902, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-01", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "into beans thrash blithely about the flu" }
 { "l_orderkey": 1061, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26288.86, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-18", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ave to slee" }
+{ "l_orderkey": 1252, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27299.97, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-22", "l_commitdate": "1997-10-10", "l_receiptdate": "1997-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "packages hag" }
+{ "l_orderkey": 1575, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39433.29, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-11-05", "l_receiptdate": "1995-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " after the unusual asym" }
 { "l_orderkey": 1733, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 41455.51, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-13", "l_commitdate": "1996-07-08", "l_receiptdate": "1996-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ess notornis. fur" }
 { "l_orderkey": 1766, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 1011.11, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1997-01-07", "l_receiptdate": "1997-02-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly blithely pending accounts. reg" }
+{ "l_orderkey": 2180, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 47522.17, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-12-08", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending, regular ideas. iron" }
+{ "l_orderkey": 2274, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23255.53, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-11-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "kly special warhorse" }
+{ "l_orderkey": 3206, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37411.07, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-10-31", "l_receiptdate": "1996-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quick theodolites hagg" }
 { "l_orderkey": 3364, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38422.18, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " slyly express" }
+{ "l_orderkey": 3456, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34377.74, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-29", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "usy pinto beans b" }
+{ "l_orderkey": 4002, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eep. quickly" }
+{ "l_orderkey": 4385, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38422.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inal frays. final, bold exc" }
+{ "l_orderkey": 4613, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e blithely against the even, bold pi" }
+{ "l_orderkey": 4674, "l_partkey": 111, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 3033.33, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " regular requests na" }
 { "l_orderkey": 4676, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 50555.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r deposits boost boldly quickly quick asymp" }
+{ "l_orderkey": 4705, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22244.42, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " fluffily pending accounts ca" }
+{ "l_orderkey": 5121, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 45499.95, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-09-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial accounts cajole ca" }
+{ "l_orderkey": 5254, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35388.85, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntegrate carefully among the pending" }
 { "l_orderkey": 5381, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48533.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-22", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "luffily spec" }
+{ "l_orderkey": 5413, "l_partkey": 111, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 36399.96, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-12", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular, regular ideas mold! final requests" }
 { "l_orderkey": 5671, "l_partkey": 111, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 42466.62, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-17", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "carefully slyly special deposit" }
-{ "l_orderkey": 1314, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual accounts slee" }
-{ "l_orderkey": 2343, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old theodolites." }
-{ "l_orderkey": 2470, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "l accounts. deposits nag daringly. express," }
-{ "l_orderkey": 3170, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-01-04", "l_receiptdate": "1998-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". express dolphins use sly" }
-{ "l_orderkey": 3588, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 37374.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xcuses sleep quickly along th" }
-{ "l_orderkey": 3779, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5050.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-07", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-02-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites. slyly regular a" }
-{ "l_orderkey": 3907, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8080.88, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "furiously final packages." }
-{ "l_orderkey": 4068, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ructions. regular, special packag" }
-{ "l_orderkey": 4261, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "into beans " }
-{ "l_orderkey": 4934, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41414.51, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "wake final, ironic f" }
-{ "l_orderkey": 1060, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ccounts. foxes maintain care" }
-{ "l_orderkey": 1991, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1992-11-29", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ckages? carefully bold depos" }
-{ "l_orderkey": 2656, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40404.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-24", "l_receiptdate": "1993-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "refully final pearls. final ideas wake. qu" }
-{ "l_orderkey": 3426, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20202.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-24", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sits cajole blit" }
-{ "l_orderkey": 3941, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29293.19, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-14", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "g the blithely" }
-{ "l_orderkey": 5575, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7070.77, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. final, final " }
-{ "l_orderkey": 2850, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30303.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1996-11-29", "l_receiptdate": "1997-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "even ideas. busy pinto beans sleep above t" }
-{ "l_orderkey": 4033, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans" }
-{ "l_orderkey": 4065, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ncies use furiously. quickly un" }
-{ "l_orderkey": 4995, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48485.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-04", "l_receiptdate": "1996-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nstructions. carefully final depos" }
-{ "l_orderkey": 5413, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22222.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-10", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits. quick" }
+{ "l_orderkey": 5731, "l_partkey": 111, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6066.66, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-02", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sits integrate slyly close platelets. quick" }
 { "l_orderkey": 98, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1010.11, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-01", "l_commitdate": "1994-12-12", "l_receiptdate": "1994-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". unusual instructions against" }
 { "l_orderkey": 1059, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26262.86, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-04-18", "l_receiptdate": "1994-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ar pinto beans at the furiously " }
+{ "l_orderkey": 1060, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-15", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ccounts. foxes maintain care" }
 { "l_orderkey": 1127, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38384.18, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-07", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ". never final packages boost acro" }
+{ "l_orderkey": 1314, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual accounts slee" }
 { "l_orderkey": 1477, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8080.88, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-25", "l_commitdate": "1997-10-18", "l_receiptdate": "1997-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ironic realms wake unusual, even ac" }
+{ "l_orderkey": 1991, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39394.29, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1992-11-29", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ckages? carefully bold depos" }
+{ "l_orderkey": 2343, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "old theodolites." }
+{ "l_orderkey": 2470, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-12", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "l accounts. deposits nag daringly. express," }
+{ "l_orderkey": 2656, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40404.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-24", "l_receiptdate": "1993-06-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "refully final pearls. final ideas wake. qu" }
 { "l_orderkey": 2785, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 37374.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-25", "l_commitdate": "1995-09-12", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tructions. furiously " }
+{ "l_orderkey": 2850, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30303.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-14", "l_commitdate": "1996-11-29", "l_receiptdate": "1997-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "even ideas. busy pinto beans sleep above t" }
+{ "l_orderkey": 3170, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-01-04", "l_receiptdate": "1998-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". express dolphins use sly" }
+{ "l_orderkey": 3426, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20202.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-24", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "sits cajole blit" }
+{ "l_orderkey": 3588, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 37374.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xcuses sleep quickly along th" }
 { "l_orderkey": 3651, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " sleep blithely furiously do" }
+{ "l_orderkey": 3779, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5050.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-07", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-02-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites. slyly regular a" }
+{ "l_orderkey": 3907, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 8.0, "l_extendedprice": 8080.88, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "furiously final packages." }
+{ "l_orderkey": 3941, "l_partkey": 110, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 29293.19, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-14", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "g the blithely" }
+{ "l_orderkey": 4033, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27272.97, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans" }
+{ "l_orderkey": 4065, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16161.76, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ncies use furiously. quickly un" }
+{ "l_orderkey": 4068, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43434.73, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-11-16", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ructions. regular, special packag" }
+{ "l_orderkey": 4261, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12121.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "into beans " }
 { "l_orderkey": 4801, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4040.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-23", "l_commitdate": "1996-04-04", "l_receiptdate": "1996-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pitaphs. regular, reg" }
+{ "l_orderkey": 4934, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41414.51, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "wake final, ironic f" }
+{ "l_orderkey": 4995, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 48485.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-04", "l_receiptdate": "1996-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nstructions. carefully final depos" }
 { "l_orderkey": 5189, "l_partkey": 110, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4040.44, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". blithely exp" }
+{ "l_orderkey": 5413, "l_partkey": 110, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 22222.42, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-10", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-11-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits. quick" }
+{ "l_orderkey": 5575, "l_partkey": 110, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7070.77, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special requests. final, final " }
+{ "l_orderkey": 5, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15136.5, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake furiously " }
+{ "l_orderkey": 99, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 36327.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-04", "l_commitdate": "1994-04-17", "l_receiptdate": "1994-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "slyly. slyly e" }
+{ "l_orderkey": 135, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 47427.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-18", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ctions wake slyly abo" }
+{ "l_orderkey": 164, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27245.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ayers wake carefully a" }
+{ "l_orderkey": 224, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " carefully. final platelets " }
+{ "l_orderkey": 449, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4036.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-27", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "are fluffily. requests are furiously" }
+{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic dependencie" }
+{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 25227.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-14", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e packages engag" }
+{ "l_orderkey": 646, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31282.1, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-01-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ag furiousl" }
+{ "l_orderkey": 896, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11100.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "rding to the pinto beans wa" }
 { "l_orderkey": 1159, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 39354.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-10-28", "l_receiptdate": "1992-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " blithely express reques" }
+{ "l_orderkey": 1410, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "to beans b" }
 { "l_orderkey": 1574, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6054.6, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "nic, final ideas snooze. " }
 { "l_orderkey": 1637, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 38345.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-05", "l_receiptdate": "1995-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "even, pending foxes nod regular" }
+{ "l_orderkey": 1798, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ld packages sleep furiously. depend" }
 { "l_orderkey": 2276, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 38345.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ans. pinto beans boost c" }
+{ "l_orderkey": 2432, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13118.3, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "arefully about the caref" }
 { "l_orderkey": 2720, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 49445.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-07-14", "l_receiptdate": "1993-07-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " accounts. fluffily bold pack" }
+{ "l_orderkey": 2976, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30273.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c ideas! unusual" }
+{ "l_orderkey": 2980, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sts. slyly regu" }
+{ "l_orderkey": 3108, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-10-01", "l_receiptdate": "1993-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " final requests. " }
+{ "l_orderkey": 3235, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9081.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l courts sleep quickly slyly " }
 { "l_orderkey": 3264, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24218.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-07", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ctions. quick" }
+{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7063.7, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pending accounts along the" }
+{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42382.2, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously final instruc" }
+{ "l_orderkey": 3492, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " unusual requests. ir" }
 { "l_orderkey": 3970, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 18163.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-06", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " maintain slyly. ir" }
+{ "l_orderkey": 4066, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 44400.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express accounts nag bli" }
+{ "l_orderkey": 4545, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8072.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " boost slyly. slyly" }
+{ "l_orderkey": 4929, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26236.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly. fl" }
+{ "l_orderkey": 5088, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10091.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans. special requests af" }
 { "l_orderkey": 5346, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7063.7, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "equests use carefully care" }
 { "l_orderkey": 5634, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 16145.6, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-15", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ess ideas are carefully pending, even re" }
 { "l_orderkey": 5636, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15136.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-21", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "efully special" }
-{ "l_orderkey": 5, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 15136.5, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake furiously " }
-{ "l_orderkey": 224, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-13", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " carefully. final platelets " }
-{ "l_orderkey": 646, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31282.1, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-01-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ag furiousl" }
-{ "l_orderkey": 896, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 11100.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "rding to the pinto beans wa" }
-{ "l_orderkey": 1410, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-04-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "to beans b" }
-{ "l_orderkey": 1798, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ld packages sleep furiously. depend" }
-{ "l_orderkey": 2980, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43391.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "sts. slyly regu" }
-{ "l_orderkey": 3108, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37336.7, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-10-01", "l_receiptdate": "1993-11-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " final requests. " }
-{ "l_orderkey": 135, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 47427.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-18", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-02-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ctions wake slyly abo" }
-{ "l_orderkey": 164, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 27245.7, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ayers wake carefully a" }
-{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-03", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ic dependencie" }
-{ "l_orderkey": 2432, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13118.3, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "arefully about the caref" }
-{ "l_orderkey": 2976, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30273.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c ideas! unusual" }
-{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 7063.7, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pending accounts along the" }
-{ "l_orderkey": 3457, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42382.2, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-06-14", "l_receiptdate": "1995-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously final instruc" }
-{ "l_orderkey": 4066, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 44400.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "express accounts nag bli" }
-{ "l_orderkey": 4545, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8072.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-01", "l_commitdate": "1993-03-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " boost slyly. slyly" }
 { "l_orderkey": 5666, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36327.6, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-03-16", "l_receiptdate": "1994-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "accounts. furiousl" }
-{ "l_orderkey": 99, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 36327.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-04", "l_commitdate": "1994-04-17", "l_receiptdate": "1994-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "slyly. slyly e" }
-{ "l_orderkey": 449, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4036.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-27", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "are fluffily. requests are furiously" }
-{ "l_orderkey": 515, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 25227.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-14", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e packages engag" }
-{ "l_orderkey": 3235, "l_partkey": 109, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9081.9, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-24", "l_receiptdate": "1995-11-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l courts sleep quickly slyly " }
-{ "l_orderkey": 3492, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 34309.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " unusual requests. ir" }
-{ "l_orderkey": 4929, "l_partkey": 109, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26236.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-06-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly. fl" }
-{ "l_orderkey": 5088, "l_partkey": 109, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10091.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans. special requests af" }
+{ "l_orderkey": 356, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48388.8, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unusual packages. furiously " }
+{ "l_orderkey": 709, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40324.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ggle fluffily carefully ironic" }
+{ "l_orderkey": 775, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20162.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en dependencies nag slowly " }
 { "l_orderkey": 871, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1996-01-24", "l_receiptdate": "1996-02-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " haggle furiou" }
-{ "l_orderkey": 2530, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-02", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial asymptotes snooze slyly regular " }
-{ "l_orderkey": 3333, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38307.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts promise bl" }
-{ "l_orderkey": 3586, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-04-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites hagg" }
-{ "l_orderkey": 5094, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23186.3, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-19", "l_receiptdate": "1993-07-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "st furiously above the fluffily care" }
-{ "l_orderkey": 5761, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " pinto beans thrash alongside of the pendi" }
 { "l_orderkey": 965, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20162.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly. carefully pending requ" }
+{ "l_orderkey": 1221, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ajole furiously. blithely expres" }
+{ "l_orderkey": 1351, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25202.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "iously regul" }
+{ "l_orderkey": 1636, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24194.4, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e carefully unusual ideas are f" }
 { "l_orderkey": 1826, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 43348.3, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-28", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-08-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss tithes use even ideas. fluffily final t" }
+{ "l_orderkey": 2179, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7056.7, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular dependencies. ironic packages haggle" }
+{ "l_orderkey": 2433, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3024.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "usly pending depos" }
+{ "l_orderkey": 2530, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-02", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ial asymptotes snooze slyly regular " }
+{ "l_orderkey": 2560, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slyly final accoun" }
 { "l_orderkey": 2561, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 39315.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-20", "l_commitdate": "1997-12-16", "l_receiptdate": "1998-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests are furiously against the" }
 { "l_orderkey": 2694, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10081.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-23", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fluffily fluffy accounts. even packages hi" }
 { "l_orderkey": 3072, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-04-22", "l_receiptdate": "1994-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " theodolites. blithely e" }
 { "l_orderkey": 3169, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6048.6, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-02-22", "l_receiptdate": "1994-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ular instructions. ca" }
+{ "l_orderkey": 3333, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38307.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ccounts promise bl" }
+{ "l_orderkey": 3586, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8064.8, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-04-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "theodolites hagg" }
 { "l_orderkey": 4354, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1994-12-05", "l_receiptdate": "1995-01-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "efully special packages use fluffily" }
 { "l_orderkey": 4357, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 17137.7, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1997-12-08", "l_receiptdate": "1998-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e carefully furiou" }
 { "l_orderkey": 4419, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45364.5, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-09-07", "l_receiptdate": "1996-08-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s doze sometimes fluffily regular a" }
 { "l_orderkey": 4480, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30243.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-06-22", "l_receiptdate": "1994-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ven braids us" }
 { "l_orderkey": 4613, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25202.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y pending platelets x-ray ironically! pend" }
-{ "l_orderkey": 356, "l_partkey": 108, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48388.8, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unusual packages. furiously " }
-{ "l_orderkey": 2179, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7056.7, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-11-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gular dependencies. ironic packages haggle" }
-{ "l_orderkey": 709, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40324.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ggle fluffily carefully ironic" }
-{ "l_orderkey": 775, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20162.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-05-22", "l_receiptdate": "1995-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en dependencies nag slowly " }
-{ "l_orderkey": 1221, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ajole furiously. blithely expres" }
-{ "l_orderkey": 1351, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 25202.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-06-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "iously regul" }
-{ "l_orderkey": 1636, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24194.4, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e carefully unusual ideas are f" }
-{ "l_orderkey": 2433, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 3024.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-08", "l_commitdate": "1994-09-24", "l_receiptdate": "1994-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "usly pending depos" }
-{ "l_orderkey": 2560, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 13105.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-07", "l_commitdate": "1992-10-21", "l_receiptdate": "1992-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slyly final accoun" }
+{ "l_orderkey": 5094, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23186.3, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-05-19", "l_receiptdate": "1993-07-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "st furiously above the fluffily care" }
 { "l_orderkey": 5316, "l_partkey": 108, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 29234.9, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-28", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ckly unusual foxes bo" }
+{ "l_orderkey": 5761, "l_partkey": 108, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36291.6, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-21", "l_receiptdate": "1998-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " pinto beans thrash alongside of the pendi" }
 { "l_orderkey": 5923, "l_partkey": 108, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2016.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "express patterns. even deposits" }
 { "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
 { "l_orderkey": 128, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 38269.8, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-10-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " cajole careful" }
-{ "l_orderkey": 390, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10071.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. final accounts x-ray beside the" }
-{ "l_orderkey": 960, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1007.1, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-26", "l_receiptdate": "1995-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y ironic packages. quickly even " }
-{ "l_orderkey": 1411, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26184.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c packages. " }
-{ "l_orderkey": 1414, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4028.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle quickly" }
-{ "l_orderkey": 3654, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20142.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s sleep about the slyly " }
-{ "l_orderkey": 3842, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24170.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "phins are quickly" }
+{ "l_orderkey": 258, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-02-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully about the fluffily silent dependencies" }
 { "l_orderkey": 354, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7049.7, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-05-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ously idly ironic accounts-- quickl" }
+{ "l_orderkey": 390, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10071.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. final accounts x-ray beside the" }
 { "l_orderkey": 450, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5035.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the pinto bea" }
 { "l_orderkey": 835, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33234.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-01", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "onic instructions among the carefully iro" }
-{ "l_orderkey": 3587, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16113.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-06-19", "l_receiptdate": "1996-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y ruthless dolphins to " }
-{ "l_orderkey": 5824, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44312.4, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-02-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily fluffily bold" }
-{ "l_orderkey": 5829, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40284.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the carefully ironic accounts. a" }
+{ "l_orderkey": 960, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1007.1, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-26", "l_receiptdate": "1995-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y ironic packages. quickly even " }
 { "l_orderkey": 1088, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30213.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "long the packages snooze careful" }
+{ "l_orderkey": 1411, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 26184.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c packages. " }
+{ "l_orderkey": 1414, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4028.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle quickly" }
 { "l_orderkey": 1477, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 32227.2, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-12", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "; quickly regula" }
 { "l_orderkey": 1509, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 17120.7, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously. blithely regular ideas haggle c" }
+{ "l_orderkey": 2150, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29205.9, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully final att" }
+{ "l_orderkey": 3587, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 16113.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-06-19", "l_receiptdate": "1996-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y ruthless dolphins to " }
+{ "l_orderkey": 3654, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 20142.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-30", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s sleep about the slyly " }
 { "l_orderkey": 3716, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42298.2, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-03", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " of the pend" }
+{ "l_orderkey": 3814, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15106.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " carefully final deposits haggle slyly" }
 { "l_orderkey": 3840, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 33234.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-10-06", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "hely silent deposits w" }
+{ "l_orderkey": 3842, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 24170.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "phins are quickly" }
+{ "l_orderkey": 4065, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ages haggle carefully" }
 { "l_orderkey": 4833, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 31220.1, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-24", "l_commitdate": "1996-07-15", "l_receiptdate": "1996-07-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ven instructions cajole against the caref" }
+{ "l_orderkey": 5380, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48340.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies haggle car" }
 { "l_orderkey": 5633, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1007.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-29", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "thely notornis: " }
 { "l_orderkey": 5729, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39276.9, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-22", "l_commitdate": "1994-11-21", "l_receiptdate": "1995-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". special pl" }
-{ "l_orderkey": 258, "l_partkey": 107, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-02-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ully about the fluffily silent dependencies" }
-{ "l_orderkey": 2150, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 29205.9, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully final att" }
-{ "l_orderkey": 3814, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 15106.5, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-03-25", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " carefully final deposits haggle slyly" }
-{ "l_orderkey": 4065, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8056.8, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ages haggle carefully" }
-{ "l_orderkey": 5380, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48340.8, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies haggle car" }
-{ "l_orderkey": 1317, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r packages impress blithely car" }
-{ "l_orderkey": 2501, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33201.3, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "leep furiously packages. even sauternes " }
-{ "l_orderkey": 2598, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12073.2, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits cajol" }
-{ "l_orderkey": 3014, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36219.6, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-28", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic r" }
-{ "l_orderkey": 3457, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages nag furiously against" }
-{ "l_orderkey": 4352, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18109.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ding to th" }
-{ "l_orderkey": 5095, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2012.2, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "detect car" }
-{ "l_orderkey": 967, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17103.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic foxes caj" }
-{ "l_orderkey": 3076, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake furiou" }
-{ "l_orderkey": 3717, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 28170.8, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts sleep q" }
-{ "l_orderkey": 4327, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40244.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-04-17", "l_receiptdate": "1995-06-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. packages are after th" }
+{ "l_orderkey": 5824, "l_partkey": 107, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 44312.4, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-02-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily fluffily bold" }
+{ "l_orderkey": 5829, "l_partkey": 107, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40284.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the carefully ironic accounts. a" }
 { "l_orderkey": 197, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 1006.1, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-15", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " even, thin dependencies sno" }
 { "l_orderkey": 229, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 29176.9, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-01-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously pending " }
 { "l_orderkey": 519, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 19115.9, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "asymptotes. p" }
+{ "l_orderkey": 967, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 17103.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-08-19", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "y ironic foxes caj" }
 { "l_orderkey": 1122, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 40244.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-07", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-02-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages sleep after the asym" }
 { "l_orderkey": 1186, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 27164.7, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-08", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "accounts. express, e" }
+{ "l_orderkey": 1283, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1006.1, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-10-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "d the sauternes. slyly ev" }
+{ "l_orderkey": 1317, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-07", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "r packages impress blithely car" }
 { "l_orderkey": 1926, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 29176.9, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-26", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "es. dependencies according to the fl" }
+{ "l_orderkey": 2501, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 33201.3, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "leep furiously packages. even sauternes " }
+{ "l_orderkey": 2598, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 12073.2, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits cajol" }
+{ "l_orderkey": 2628, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44268.4, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-14", "l_receiptdate": "1994-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lyly final, pending ide" }
 { "l_orderkey": 2628, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 14085.4, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1993-11-30", "l_receiptdate": "1994-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the furiously unusual pi" }
+{ "l_orderkey": 3014, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 36219.6, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-28", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-01-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "iously ironic r" }
+{ "l_orderkey": 3076, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake furiou" }
+{ "l_orderkey": 3457, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22134.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages nag furiously against" }
 { "l_orderkey": 3520, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5030.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-12-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly even ideas haggle " }
+{ "l_orderkey": 3681, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lyly special pinto " }
+{ "l_orderkey": 3717, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 28170.8, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-25", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts sleep q" }
+{ "l_orderkey": 4001, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 26158.6, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-26", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate blithely" }
+{ "l_orderkey": 4327, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 40244.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-04-17", "l_receiptdate": "1995-06-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. packages are after th" }
+{ "l_orderkey": 4352, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 18109.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ding to th" }
 { "l_orderkey": 4836, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15091.5, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-03-14", "l_receiptdate": "1997-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eep slyly. even requests cajole" }
+{ "l_orderkey": 5095, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2012.2, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-09", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "detect car" }
 { "l_orderkey": 5155, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 28170.8, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s cajole. accounts wake. thinly quiet pla" }
 { "l_orderkey": 5351, "l_partkey": 106, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2012.2, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-12", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-05-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "g accounts wake furiously slyly even dolph" }
 { "l_orderkey": 5632, "l_partkey": 106, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21128.1, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-03-10", "l_receiptdate": "1996-04-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "refully regular pinto beans. ironic reques" }
-{ "l_orderkey": 1283, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1006.1, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-10-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "d the sauternes. slyly ev" }
-{ "l_orderkey": 2628, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44268.4, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-14", "l_receiptdate": "1994-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "lyly final, pending ide" }
-{ "l_orderkey": 3681, "l_partkey": 106, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35213.5, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-05-18", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lyly special pinto " }
-{ "l_orderkey": 4001, "l_partkey": 106, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 26158.6, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-26", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tegrate blithely" }
+{ "l_orderkey": 69, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s sleep carefully bold, " }
+{ "l_orderkey": 515, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar deposits th" }
+{ "l_orderkey": 615, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36183.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-06-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. carefully final pinto bea" }
+{ "l_orderkey": 806, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1005.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ar accounts? pending, pending foxes a" }
+{ "l_orderkey": 992, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-15", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nic instructions n" }
+{ "l_orderkey": 1027, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ilent, express foxes near the blithely sp" }
+{ "l_orderkey": 1189, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e regular deposits. quickly quiet deposi" }
+{ "l_orderkey": 1728, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23117.3, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-09-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ns. pending, final ac" }
+{ "l_orderkey": 1859, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12061.2, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es. unusual, silent request" }
 { "l_orderkey": 1986, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-14", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "yly into the carefully even " }
 { "l_orderkey": 2082, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 12061.2, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-02-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " ironic instructions. carefull" }
 { "l_orderkey": 2086, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 44224.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-30", "l_receiptdate": "1994-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "latelets s" }
-{ "l_orderkey": 2593, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37188.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1993-10-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake bravel" }
-{ "l_orderkey": 3648, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " above the somas boost furious" }
-{ "l_orderkey": 4288, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39198.9, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-25", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uffy theodolites run" }
-{ "l_orderkey": 5382, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 48244.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-19", "l_receiptdate": "1992-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nts integrate quickly ca" }
-{ "l_orderkey": 615, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36183.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-06-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. carefully final pinto bea" }
-{ "l_orderkey": 1189, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e regular deposits. quickly quiet deposi" }
 { "l_orderkey": 2466, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-04-06", "l_receiptdate": "1994-06-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sly regular deposits. regular, regula" }
-{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34173.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "pendencies!" }
-{ "l_orderkey": 5281, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38193.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-17", "l_commitdate": "1995-12-19", "l_receiptdate": "1996-02-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n asymptotes could wake about th" }
-{ "l_orderkey": 515, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-03", "l_receiptdate": "1993-10-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar deposits th" }
-{ "l_orderkey": 992, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-15", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nic instructions n" }
-{ "l_orderkey": 1728, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23117.3, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-09-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ns. pending, final ac" }
-{ "l_orderkey": 1859, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 12061.2, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es. unusual, silent request" }
+{ "l_orderkey": 2593, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 37188.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1993-10-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake bravel" }
 { "l_orderkey": 2596, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " instructions shall have" }
-{ "l_orderkey": 2791, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8040.8, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-24", "l_receiptdate": "1995-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se. close ideas alongs" }
-{ "l_orderkey": 3042, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "the requests detect fu" }
-{ "l_orderkey": 3778, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 49249.9, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ans. furiously " }
-{ "l_orderkey": 3809, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46234.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l asymptotes. special " }
-{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19096.9, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nt dependencies. furiously regular ideas d" }
-{ "l_orderkey": 69, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 32163.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s sleep carefully bold, " }
-{ "l_orderkey": 806, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1005.1, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ar accounts? pending, pending foxes a" }
-{ "l_orderkey": 1027, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 10051.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ilent, express foxes near the blithely sp" }
 { "l_orderkey": 2756, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 31158.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "en instructions use quickly." }
+{ "l_orderkey": 2791, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 8040.8, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-30", "l_commitdate": "1994-11-24", "l_receiptdate": "1995-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "se. close ideas alongs" }
 { "l_orderkey": 2850, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49249.9, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " slyly unusual req" }
 { "l_orderkey": 3008, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31158.1, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-01", "l_commitdate": "1996-01-20", "l_receiptdate": "1995-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nts use thinly around the carefully iro" }
+{ "l_orderkey": 3042, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "the requests detect fu" }
 { "l_orderkey": 3169, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26132.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-03-21", "l_receiptdate": "1994-04-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ter the regular ideas. slyly iro" }
 { "l_orderkey": 3335, "l_partkey": 105, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 13066.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-20", "l_commitdate": "1995-12-20", "l_receiptdate": "1996-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "out the special asymptotes" }
+{ "l_orderkey": 3648, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-06", "l_receiptdate": "1993-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " above the somas boost furious" }
+{ "l_orderkey": 3778, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 49.0, "l_extendedprice": 49249.9, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ans. furiously " }
+{ "l_orderkey": 3809, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46234.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l asymptotes. special " }
 { "l_orderkey": 3845, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30153.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "counts do wake blithely. ironic requests " }
 { "l_orderkey": 3969, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 4020.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "dencies wake blithely? quickly even theodo" }
+{ "l_orderkey": 4288, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 39198.9, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-25", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uffy theodolites run" }
+{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19096.9, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nt dependencies. furiously regular ideas d" }
+{ "l_orderkey": 4294, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34173.4, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-10-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "pendencies!" }
 { "l_orderkey": 4900, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 40204.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-07-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "luffily final dol" }
 { "l_orderkey": 5152, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9045.9, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-11", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " cajole furiously alongside of the bo" }
+{ "l_orderkey": 5281, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38193.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-17", "l_commitdate": "1995-12-19", "l_receiptdate": "1996-02-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "n asymptotes could wake about th" }
+{ "l_orderkey": 5382, "l_partkey": 105, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 48.0, "l_extendedprice": 48244.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-19", "l_receiptdate": "1992-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nts integrate quickly ca" }
 { "l_orderkey": 5410, "l_partkey": 105, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 41209.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-25", "l_commitdate": "1998-10-20", "l_receiptdate": "1998-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "sly. slyly ironic theodolites" }
 { "l_orderkey": 5731, "l_partkey": 105, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 11056.1, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-06", "l_commitdate": "1997-07-08", "l_receiptdate": "1997-06-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously final accounts wake. d" }
 { "l_orderkey": 71, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 39159.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l accounts sleep across the pack" }
 { "l_orderkey": 133, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 27110.7, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-21", "l_commitdate": "1998-02-23", "l_receiptdate": "1997-12-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yly even gifts after the sl" }
+{ "l_orderkey": 1538, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28114.8, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "bout the fluffily unusual" }
 { "l_orderkey": 1697, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24098.4, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-29", "l_commitdate": "1996-12-19", "l_receiptdate": "1997-01-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ts cajole carefully above the carefully" }
-{ "l_orderkey": 1862, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26106.6, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g carefully: thinly ironic deposits af" }
-{ "l_orderkey": 2598, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4016.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " across the furiously fi" }
-{ "l_orderkey": 4897, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19077.9, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! ironic, pending dependencies doze furiou" }
-{ "l_orderkey": 5377, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic, final" }
-{ "l_orderkey": 5409, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38155.8, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-03-29", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic, regular accounts! blithely even" }
 { "l_orderkey": 1829, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49200.9, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-26", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ound the quickly " }
+{ "l_orderkey": 1862, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26106.6, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g carefully: thinly ironic deposits af" }
+{ "l_orderkey": 2179, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5020.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-10-08", "l_receiptdate": "1996-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts haggle blithely. ironic, careful theodol" }
 { "l_orderkey": 2246, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43176.3, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ainst the ironic theodolites haggle fi" }
 { "l_orderkey": 2469, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 49200.9, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-03-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " requests are car" }
+{ "l_orderkey": 2598, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4016.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-23", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-05-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " across the furiously fi" }
 { "l_orderkey": 3010, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 9036.9, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-18", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "inal packages. quickly even pinto" }
-{ "l_orderkey": 3488, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48196.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-29", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sly? final requests " }
-{ "l_orderkey": 1538, "l_partkey": 104, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 28114.8, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "bout the fluffily unusual" }
-{ "l_orderkey": 2179, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5020.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-10-08", "l_receiptdate": "1996-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ts haggle blithely. ironic, careful theodol" }
-{ "l_orderkey": 3748, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "old reques" }
 { "l_orderkey": 3047, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 17069.7, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-14", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic instruction" }
+{ "l_orderkey": 3488, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 48196.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-29", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sly? final requests " }
+{ "l_orderkey": 3748, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-04-15", "l_receiptdate": "1998-05-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "old reques" }
+{ "l_orderkey": 4897, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19077.9, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-08", "l_commitdate": "1992-12-14", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "! ironic, pending dependencies doze furiou" }
 { "l_orderkey": 5221, "l_partkey": 104, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24098.4, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-04", "l_commitdate": "1995-08-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s pinto beans sleep. sly" }
+{ "l_orderkey": 5377, "l_partkey": 104, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12049.2, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic, final" }
+{ "l_orderkey": 5409, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 38155.8, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-03-29", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic, regular accounts! blithely even" }
 { "l_orderkey": 5479, "l_partkey": 104, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 19077.9, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully bo" }
-{ "l_orderkey": 161, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19058.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", regular sheaves sleep along" }
-{ "l_orderkey": 832, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45139.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "foxes engage slyly alon" }
-{ "l_orderkey": 2017, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49151.9, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-01", "l_receiptdate": "1998-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " after the unusual instructions. sly" }
-{ "l_orderkey": 4484, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "the ironic, final theodo" }
-{ "l_orderkey": 4515, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-28", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-04-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ding instructions again" }
-{ "l_orderkey": 5377, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " silent wa" }
 { "l_orderkey": 68, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 30093.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-07-11", "l_receiptdate": "1998-08-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oxes are slyly blithely fin" }
-{ "l_orderkey": 1504, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " accounts sleep. furiou" }
-{ "l_orderkey": 2661, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-04-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " foxes affix quickly ironic request" }
-{ "l_orderkey": 3136, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7021.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-09-14", "l_receiptdate": "1994-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic pinto beans are slyly. f" }
-{ "l_orderkey": 4422, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39120.9, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-24", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "en hockey players engage" }
-{ "l_orderkey": 4932, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15046.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-15", "l_commitdate": "1993-10-25", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly. unusu" }
+{ "l_orderkey": 161, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 19058.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ", regular sheaves sleep along" }
 { "l_orderkey": 353, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 39120.9, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-15", "l_commitdate": "1994-03-30", "l_receiptdate": "1994-02-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "losely quickly even accounts. c" }
 { "l_orderkey": 711, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27083.7, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-10-26", "l_receiptdate": "1993-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "slyly. ironic asy" }
+{ "l_orderkey": 832, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 45139.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "foxes engage slyly alon" }
+{ "l_orderkey": 1504, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " accounts sleep. furiou" }
 { "l_orderkey": 1956, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 16049.6, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es cajole blithely. pen" }
+{ "l_orderkey": 2017, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49151.9, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-01", "l_receiptdate": "1998-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " after the unusual instructions. sly" }
 { "l_orderkey": 2209, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10031.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "players. carefully reg" }
-{ "l_orderkey": 4869, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24074.4, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "se deposits above the sly, q" }
-{ "l_orderkey": 4900, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly final acco" }
-{ "l_orderkey": 5445, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-09-15", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "old depend" }
-{ "l_orderkey": 5827, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-16", "l_commitdate": "1998-09-14", "l_receiptdate": "1998-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ans. furiously special instruct" }
 { "l_orderkey": 2400, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 48148.8, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-07", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fore the car" }
+{ "l_orderkey": 2661, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 22068.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-04-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " foxes affix quickly ironic request" }
+{ "l_orderkey": 3136, "l_partkey": 103, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 7021.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-09-14", "l_receiptdate": "1994-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic pinto beans are slyly. f" }
 { "l_orderkey": 3205, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38117.8, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "usly quiet accounts. slyly pending pinto " }
 { "l_orderkey": 3936, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 26080.6, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "quickly pen" }
+{ "l_orderkey": 4422, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 39120.9, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-02", "l_commitdate": "1995-06-24", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "en hockey players engage" }
+{ "l_orderkey": 4484, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "the ironic, final theodo" }
+{ "l_orderkey": 4515, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50155.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-28", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-04-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ding instructions again" }
+{ "l_orderkey": 4869, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 24074.4, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "se deposits above the sly, q" }
 { "l_orderkey": 4900, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 48148.8, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-18", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uickly ironic ideas kindle s" }
+{ "l_orderkey": 4900, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-07-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly final acco" }
 { "l_orderkey": 4931, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 8024.8, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "dependencies are slyly" }
-{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46096.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly bold deposits cajole according to" }
-{ "l_orderkey": 2305, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-02", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-04-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " haggle caref" }
-{ "l_orderkey": 2854, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7014.7, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-08-14", "l_receiptdate": "1994-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the pending" }
-{ "l_orderkey": 5408, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cross the dolphins h" }
-{ "l_orderkey": 5696, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-15", "l_receiptdate": "1995-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "n patterns lose slyly fina" }
+{ "l_orderkey": 4932, "l_partkey": 103, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 15046.5, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-15", "l_commitdate": "1993-10-25", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly. unusu" }
+{ "l_orderkey": 5377, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-26", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " silent wa" }
+{ "l_orderkey": 5445, "l_partkey": 103, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 46142.6, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-09-15", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "old depend" }
+{ "l_orderkey": 5827, "l_partkey": 103, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 23071.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-16", "l_commitdate": "1998-09-14", "l_receiptdate": "1998-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ans. furiously special instruct" }
+{ "l_orderkey": 167, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-19", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sly during the u" }
+{ "l_orderkey": 198, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 33069.3, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ending foxes acr" }
 { "l_orderkey": 674, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-25", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-11-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ve the quickly even deposits. blithe" }
+{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46096.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly bold deposits cajole according to" }
+{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24050.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix slyly. furiously i" }
+{ "l_orderkey": 897, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "into beans. slyly special fox" }
 { "l_orderkey": 929, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 7014.7, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-12-19", "l_receiptdate": "1993-01-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ithely. slyly c" }
+{ "l_orderkey": 1538, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uses maintain blithely. fluffily" }
 { "l_orderkey": 1958, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4008.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-12-09", "l_receiptdate": "1995-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he slyly even dependencies " }
 { "l_orderkey": 2119, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 36075.6, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly bold foxes. ironic accoun" }
+{ "l_orderkey": 2305, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-02", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-04-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " haggle caref" }
 { "l_orderkey": 2662, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43090.3, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-11-04", "l_receiptdate": "1996-12-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". slyly specia" }
 { "l_orderkey": 2693, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43090.3, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "as are according to th" }
+{ "l_orderkey": 2854, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7014.7, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-08-14", "l_receiptdate": "1994-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the pending" }
 { "l_orderkey": 2919, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-28", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-01-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "re slyly. regular ideas detect furiousl" }
 { "l_orderkey": 2919, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 44092.4, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-02-03", "l_receiptdate": "1994-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es doze around the furiously " }
+{ "l_orderkey": 3042, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-02", "l_receiptdate": "1994-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ng the furiously r" }
 { "l_orderkey": 3527, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 47098.7, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-14", "l_commitdate": "1997-07-29", "l_receiptdate": "1997-07-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "unts. express re" }
+{ "l_orderkey": 3844, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5010.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-05-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unwind quickly about the pending, i" }
 { "l_orderkey": 4007, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 5010.5, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y unusual packa" }
+{ "l_orderkey": 4032, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8016.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ometimes even cou" }
+{ "l_orderkey": 4131, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47098.7, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges. ironic pinto be" }
 { "l_orderkey": 4166, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ily ironic deposits print furiously. iron" }
+{ "l_orderkey": 4322, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10021.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular ideas engage carefully quick" }
+{ "l_orderkey": 4453, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26054.6, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-05-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "express packages are" }
 { "l_orderkey": 4835, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-05", "l_commitdate": "1995-01-04", "l_receiptdate": "1995-02-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e carefully regular foxes. deposits are sly" }
 { "l_orderkey": 4866, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1002.1, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-01", "l_receiptdate": "1997-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "latelets nag. q" }
+{ "l_orderkey": 5030, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22046.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". quickly regular foxes believe" }
+{ "l_orderkey": 5217, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-15", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "pending packages cajole ne" }
+{ "l_orderkey": 5408, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cross the dolphins h" }
 { "l_orderkey": 5415, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 44092.4, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-10-26", "l_receiptdate": "1992-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " requests. unusual theodolites sleep agains" }
 { "l_orderkey": 5415, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ges around the fur" }
 { "l_orderkey": 5540, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nic asymptotes could hav" }
-{ "l_orderkey": 5762, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27056.7, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-03-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the bold ideas. carefully sp" }
-{ "l_orderkey": 742, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 24050.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-03-12", "l_receiptdate": "1995-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix slyly. furiously i" }
-{ "l_orderkey": 1538, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32067.2, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uses maintain blithely. fluffily" }
-{ "l_orderkey": 4131, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 47098.7, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges. ironic pinto be" }
-{ "l_orderkey": 4322, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 10021.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular ideas engage carefully quick" }
-{ "l_orderkey": 5030, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 22046.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": ". quickly regular foxes believe" }
-{ "l_orderkey": 5217, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23048.3, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-15", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-11-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "pending packages cajole ne" }
-{ "l_orderkey": 167, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-19", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "sly during the u" }
-{ "l_orderkey": 198, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 33069.3, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ending foxes acr" }
-{ "l_orderkey": 897, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2004.2, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "into beans. slyly special fox" }
-{ "l_orderkey": 3042, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 28058.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-02", "l_receiptdate": "1994-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ng the furiously r" }
-{ "l_orderkey": 3844, "l_partkey": 102, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5010.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-05-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unwind quickly about the pending, i" }
-{ "l_orderkey": 4032, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8016.8, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ometimes even cou" }
-{ "l_orderkey": 4453, "l_partkey": 102, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 26054.6, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-06-07", "l_receiptdate": "1997-05-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "express packages are" }
 { "l_orderkey": 5633, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10021.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-15", "l_commitdate": "1998-08-03", "l_receiptdate": "1998-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its cajole fluffily fluffily special pinto" }
+{ "l_orderkey": 5696, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 6012.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-07-15", "l_receiptdate": "1995-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "n patterns lose slyly fina" }
+{ "l_orderkey": 5762, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 27056.7, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-03-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "across the bold ideas. carefully sp" }
 { "l_orderkey": 5792, "l_partkey": 102, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31065.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-17", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s? furiously even instructions " }
 { "l_orderkey": 5984, "l_partkey": 102, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25052.5, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-07-21", "l_receiptdate": "1994-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular accounts. even packages nag slyly" }
+{ "l_orderkey": 420, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5005.5, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-04", "l_commitdate": "1996-01-02", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole blit" }
+{ "l_orderkey": 581, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49053.9, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly regular pinto beans acr" }
+{ "l_orderkey": 644, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-07-26", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "iously ironic pinto beans. bold packa" }
+{ "l_orderkey": 742, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48052.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-04-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " platelets " }
 { "l_orderkey": 1123, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 38041.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " blithely carefully unusual reques" }
+{ "l_orderkey": 1186, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ely alongside of the blithel" }
 { "l_orderkey": 1571, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 48052.8, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-28", "l_commitdate": "1993-01-04", "l_receiptdate": "1993-01-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly pending p" }
 { "l_orderkey": 1893, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 18019.8, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-24", "l_commitdate": "1998-01-12", "l_receiptdate": "1998-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "g packages. fluffily final reques" }
-{ "l_orderkey": 2048, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12013.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " even theodoli" }
-{ "l_orderkey": 2599, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " express accoun" }
-{ "l_orderkey": 4673, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-10-31", "l_receiptdate": "1997-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " gifts cajole dari" }
-{ "l_orderkey": 5062, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9009.9, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " silent theodolites wake. c" }
-{ "l_orderkey": 5572, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14015.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he fluffily express packages. fluffily fina" }
-{ "l_orderkey": 5764, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28030.8, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sleep furi" }
-{ "l_orderkey": 644, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-07-26", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "iously ironic pinto beans. bold packa" }
 { "l_orderkey": 1958, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 31034.1, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-11-12", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "r deposits c" }
-{ "l_orderkey": 2371, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "requests. regular pinto beans wake. car" }
-{ "l_orderkey": 4355, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 47051.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-29", "l_receiptdate": "1997-01-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e. realms integrate " }
-{ "l_orderkey": 5126, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1992-12-19", "l_receiptdate": "1993-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e silently. ironic, unusual accounts" }
-{ "l_orderkey": 581, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 49053.9, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". slyly regular pinto beans acr" }
-{ "l_orderkey": 742, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48052.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-01-23", "l_receiptdate": "1995-04-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " platelets " }
-{ "l_orderkey": 1186, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-20", "l_commitdate": "1996-10-23", "l_receiptdate": "1996-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ely alongside of the blithel" }
 { "l_orderkey": 1990, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 46050.6, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-03-14", "l_receiptdate": "1995-01-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar sentiments." }
-{ "l_orderkey": 3905, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uses are care" }
-{ "l_orderkey": 4965, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27029.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "efully final foxes" }
-{ "l_orderkey": 420, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5005.5, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-04", "l_commitdate": "1996-01-02", "l_receiptdate": "1995-11-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole blit" }
+{ "l_orderkey": 2048, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 12013.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " even theodoli" }
 { "l_orderkey": 2053, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-25", "l_commitdate": "1995-04-12", "l_receiptdate": "1995-05-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly ironic foxes haggle slyly speci" }
 { "l_orderkey": 2147, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 4004.4, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "mong the blithely special" }
+{ "l_orderkey": 2371, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "requests. regular pinto beans wake. car" }
 { "l_orderkey": 2464, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 20022.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-01-02", "l_receiptdate": "1998-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sts. slyly close ideas shall h" }
+{ "l_orderkey": 2599, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11012.1, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " express accoun" }
 { "l_orderkey": 3044, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 10011.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly ironic requests. s" }
 { "l_orderkey": 3683, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 35038.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " the furiously expr" }
+{ "l_orderkey": 3905, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "uses are care" }
+{ "l_orderkey": 4355, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 47051.7, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-29", "l_receiptdate": "1997-01-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e. realms integrate " }
 { "l_orderkey": 4390, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 42046.2, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-16", "l_receiptdate": "1995-06-17", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "arefully even accoun" }
+{ "l_orderkey": 4673, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 44048.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-10-31", "l_receiptdate": "1997-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " gifts cajole dari" }
+{ "l_orderkey": 4965, "l_partkey": 101, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 27029.7, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "efully final foxes" }
+{ "l_orderkey": 5062, "l_partkey": 101, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9009.9, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1992-12-01", "l_receiptdate": "1993-01-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " silent theodolites wake. c" }
+{ "l_orderkey": 5126, "l_partkey": 101, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43047.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1992-12-19", "l_receiptdate": "1993-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e silently. ironic, unusual accounts" }
+{ "l_orderkey": 5572, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 14015.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-11-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "he fluffily express packages. fluffily fina" }
 { "l_orderkey": 5665, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 32035.2, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-08-01", "l_receiptdate": "1993-09-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "f the slyly even requests! regular request" }
-{ "l_orderkey": 292, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " bold, pending theodolites u" }
-{ "l_orderkey": 675, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15001.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits after the furio" }
-{ "l_orderkey": 1606, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ously final requests. slowly ironic ex" }
-{ "l_orderkey": 2022, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36003.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly after the foxes. regular, final inst" }
-{ "l_orderkey": 2785, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34003.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly final packages haggl" }
-{ "l_orderkey": 2852, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12001.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "le. request" }
-{ "l_orderkey": 3649, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-20", "l_receiptdate": "1994-07-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "c accounts. quickly final theodo" }
-{ "l_orderkey": 4134, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ironic pin" }
-{ "l_orderkey": 4738, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the blithely ironic braids sleep slyly" }
-{ "l_orderkey": 4839, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19001.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits sublate furiously ir" }
-{ "l_orderkey": 4961, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10001.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests. regular, ironic ideas at the ironi" }
-{ "l_orderkey": 5799, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " furiously s" }
-{ "l_orderkey": 1027, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13001.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ily ironic ideas use" }
-{ "l_orderkey": 1223, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28002.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " quickly ironic requests. furious" }
-{ "l_orderkey": 2470, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-01", "l_receiptdate": "1997-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages " }
-{ "l_orderkey": 3683, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-07-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "xpress accounts sleep slyly re" }
-{ "l_orderkey": 4739, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly even packages use across th" }
-{ "l_orderkey": 4928, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4000.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bout the slyly final accounts. carefull" }
-{ "l_orderkey": 5509, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-24", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "counts sleep. f" }
-{ "l_orderkey": 5633, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48004.8, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even courts haggle slyly at the requ" }
+{ "l_orderkey": 5764, "l_partkey": 101, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28030.8, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sleep furi" }
 { "l_orderkey": 166, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 41004.1, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "hily along the blithely pending fo" }
+{ "l_orderkey": 292, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-03-06", "l_receiptdate": "1992-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " bold, pending theodolites u" }
+{ "l_orderkey": 641, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1000.1, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " nag across the regular foxes." }
+{ "l_orderkey": 675, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 15001.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "posits after the furio" }
 { "l_orderkey": 773, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5000.5, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-12-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar requests. regular, thin packages u" }
+{ "l_orderkey": 930, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21002.1, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "foxes. regular deposits integrate carefu" }
 { "l_orderkey": 933, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 26002.6, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-09", "l_commitdate": "1992-11-03", "l_receiptdate": "1992-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " the deposits affix slyly after t" }
+{ "l_orderkey": 1027, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 13001.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-22", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ily ironic ideas use" }
 { "l_orderkey": 1028, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 8000.8, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-03-28", "l_receiptdate": "1994-02-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e carefully final packages. furiously fi" }
 { "l_orderkey": 1152, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 25002.5, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-20", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "efully ironic accounts. sly instructions wa" }
-{ "l_orderkey": 1828, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33003.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s boost carefully. pending d" }
-{ "l_orderkey": 1890, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43004.3, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p ironic, express accounts. fu" }
-{ "l_orderkey": 2567, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 32003.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-24", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even, iro" }
-{ "l_orderkey": 3777, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11001.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-06-05", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ld ideas. even theodolites" }
-{ "l_orderkey": 5920, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42004.2, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lar, ironic dependencies sno" }
-{ "l_orderkey": 5954, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35003.5, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tions maintain slyly. furious" }
-{ "l_orderkey": 641, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 1000.1, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-03", "l_commitdate": "1993-10-28", "l_receiptdate": "1993-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " nag across the regular foxes." }
-{ "l_orderkey": 930, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 21002.1, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-16", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "foxes. regular deposits integrate carefu" }
+{ "l_orderkey": 1223, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 28002.8, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " quickly ironic requests. furious" }
 { "l_orderkey": 1445, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-02-22", "l_receiptdate": "1995-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al accounts use furiously a" }
+{ "l_orderkey": 1606, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-06-26", "l_receiptdate": "1997-04-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ously final requests. slowly ironic ex" }
+{ "l_orderkey": 1828, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 33003.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s boost carefully. pending d" }
 { "l_orderkey": 1857, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 41004.1, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " the slyly" }
+{ "l_orderkey": 1890, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 43004.3, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p ironic, express accounts. fu" }
+{ "l_orderkey": 2022, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 36003.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-24", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly after the foxes. regular, final inst" }
+{ "l_orderkey": 2470, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-06-01", "l_receiptdate": "1997-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages " }
+{ "l_orderkey": 2567, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 32003.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-24", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even, iro" }
+{ "l_orderkey": 2785, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 34003.4, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-09-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly final packages haggl" }
+{ "l_orderkey": 2852, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 12001.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-25", "l_commitdate": "1993-03-24", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "le. request" }
 { "l_orderkey": 3170, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 21002.1, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-09", "l_commitdate": "1998-01-31", "l_receiptdate": "1997-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "o beans. carefully final requests dou" }
 { "l_orderkey": 3233, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 2000.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1995-01-02", "l_receiptdate": "1995-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " across the bold packages" }
 { "l_orderkey": 3461, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 49004.9, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-16", "l_receiptdate": "1993-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ual request" }
+{ "l_orderkey": 3649, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 24002.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-07", "l_commitdate": "1994-08-20", "l_receiptdate": "1994-07-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "c accounts. quickly final theodo" }
+{ "l_orderkey": 3683, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 23002.3, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-07-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "xpress accounts sleep slyly re" }
+{ "l_orderkey": 3777, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 11001.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-06-05", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ld ideas. even theodolites" }
 { "l_orderkey": 3808, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 34003.4, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-13", "l_commitdate": "1994-07-22", "l_receiptdate": "1994-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " pearls will have to " }
+{ "l_orderkey": 4134, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-03-27", "l_receiptdate": "1995-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ironic pin" }
 { "l_orderkey": 4262, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 28002.8, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-09-06", "l_receiptdate": "1996-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ironic, regular depend" }
+{ "l_orderkey": 4738, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 50005.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the blithely ironic braids sleep slyly" }
+{ "l_orderkey": 4739, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-04-12", "l_receiptdate": "1993-06-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly even packages use across th" }
 { "l_orderkey": 4742, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-12", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-06-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ke carefully. do" }
+{ "l_orderkey": 4839, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 19001.9, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-07-14", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " deposits sublate furiously ir" }
+{ "l_orderkey": 4928, "l_partkey": 100, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4000.4, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-25", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-11-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "bout the slyly final accounts. carefull" }
+{ "l_orderkey": 4961, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 10001.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests. regular, ironic ideas at the ironi" }
+{ "l_orderkey": 5509, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 45004.5, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-24", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "counts sleep. f" }
+{ "l_orderkey": 5633, "l_partkey": 100, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 48004.8, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even courts haggle slyly at the requ" }
+{ "l_orderkey": 5799, "l_partkey": 100, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 30003.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-09-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " furiously s" }
+{ "l_orderkey": 5920, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 42004.2, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lar, ironic dependencies sno" }
+{ "l_orderkey": 5954, "l_partkey": 100, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 35003.5, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-02-06", "l_receiptdate": "1993-04-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tions maintain slyly. furious" }
 { "l_orderkey": 197, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38964.51, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-07-01", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "press accounts. daringly sp" }
-{ "l_orderkey": 1409, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22979.07, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ions. slyly ironic packages wake quick" }
-{ "l_orderkey": 1893, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42960.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "he carefully regular " }
-{ "l_orderkey": 2599, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 28973.61, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly express dolphins. special, " }
-{ "l_orderkey": 2982, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12988.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "regular deposits unwind alongside " }
-{ "l_orderkey": 4290, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2997.27, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-03-07", "l_receiptdate": "1995-04-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lar platelets cajole" }
-{ "l_orderkey": 5447, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30971.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes sleep. blithely unusual accounts det" }
-{ "l_orderkey": 646, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33969.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "slow accounts. fluffily idle instructions" }
-{ "l_orderkey": 3586, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " ironic pinto beans cajole carefully theo" }
-{ "l_orderkey": 4998, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-03", "l_receiptdate": "1992-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions nag quickly according to the theodolit" }
-{ "l_orderkey": 5509, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16984.53, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts wake ar" }
-{ "l_orderkey": 5921, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43959.96, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ain about the special" }
 { "l_orderkey": 259, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13987.26, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ons against the express acco" }
 { "l_orderkey": 288, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35967.24, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-22", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "yly pending excu" }
+{ "l_orderkey": 646, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33969.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "slow accounts. fluffily idle instructions" }
 { "l_orderkey": 803, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20980.89, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-25", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ironic packages cajole slyly. un" }
 { "l_orderkey": 1251, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 36966.33, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-12-01", "l_receiptdate": "1998-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "finally bold requests" }
-{ "l_orderkey": 2823, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17983.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "eas. decoys cajole deposi" }
-{ "l_orderkey": 3303, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 36966.33, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-16", "l_commitdate": "1998-03-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " carefully ironic asympt" }
+{ "l_orderkey": 1409, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22979.07, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-02-25", "l_receiptdate": "1993-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ions. slyly ironic packages wake quick" }
 { "l_orderkey": 1732, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35967.24, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-04-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ve the accounts. slowly ironic multip" }
+{ "l_orderkey": 1893, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42960.87, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-25", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "he carefully regular " }
 { "l_orderkey": 2149, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9990.9, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "eposits sleep above" }
+{ "l_orderkey": 2599, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 28973.61, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-10", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly express dolphins. special, " }
+{ "l_orderkey": 2823, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17983.62, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-10-30", "l_receiptdate": "1995-11-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "eas. decoys cajole deposi" }
+{ "l_orderkey": 2982, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12988.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "regular deposits unwind alongside " }
+{ "l_orderkey": 3303, "l_partkey": 99, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 36966.33, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-16", "l_commitdate": "1998-03-07", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " carefully ironic asympt" }
 { "l_orderkey": 3462, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1998.18, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-10", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nic packages. even accounts alongside " }
+{ "l_orderkey": 3586, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-04-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " ironic pinto beans cajole carefully theo" }
 { "l_orderkey": 4002, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3996.36, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ccording to the careful" }
+{ "l_orderkey": 4290, "l_partkey": 99, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2997.27, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-25", "l_commitdate": "1995-03-07", "l_receiptdate": "1995-04-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lar platelets cajole" }
+{ "l_orderkey": 4998, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7992.72, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-03", "l_receiptdate": "1992-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ions nag quickly according to the theodolit" }
+{ "l_orderkey": 5447, "l_partkey": 99, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30971.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-07-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes sleep. blithely unusual accounts det" }
+{ "l_orderkey": 5509, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16984.53, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-01", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts wake ar" }
+{ "l_orderkey": 5921, "l_partkey": 99, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43959.96, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ain about the special" }
 { "l_orderkey": 192, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-05", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly pending theodolites haggle quickly fluf" }
-{ "l_orderkey": 1159, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-09", "l_commitdate": "1992-12-07", "l_receiptdate": "1992-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "h furiousl" }
-{ "l_orderkey": 2883, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-03", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even requests cajole. special, regular " }
-{ "l_orderkey": 4932, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "as. special depende" }
-{ "l_orderkey": 5603, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49904.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-06", "l_commitdate": "1992-08-20", "l_receiptdate": "1992-10-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "final theodolites accor" }
 { "l_orderkey": 678, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15969.44, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-04-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " about the " }
-{ "l_orderkey": 864, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6986.63, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-07", "l_receiptdate": "1997-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven requests should sleep along " }
-{ "l_orderkey": 1348, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1996.18, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final packages use fluffily express ac" }
-{ "l_orderkey": 1511, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28944.61, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s cajole furiously against " }
-{ "l_orderkey": 4390, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31938.88, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-15", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions across" }
-{ "l_orderkey": 4421, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36929.33, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l accounts. ironic request" }
-{ "l_orderkey": 4960, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7984.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-14", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "as. busily regular packages nag. " }
-{ "l_orderkey": 5696, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19961.8, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "silent, pending ideas sleep fluffil" }
 { "l_orderkey": 736, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13973.26, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-10-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nstructions." }
+{ "l_orderkey": 864, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6986.63, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-10-07", "l_receiptdate": "1997-12-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven requests should sleep along " }
 { "l_orderkey": 963, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47908.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ages. quickly express deposits cajole pe" }
-{ "l_orderkey": 2561, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4990.45, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p ironic, regular pinto beans." }
-{ "l_orderkey": 3463, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42917.87, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " across the " }
-{ "l_orderkey": 3617, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15969.44, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " slyly on th" }
-{ "l_orderkey": 3778, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-02", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits. theodol" }
-{ "l_orderkey": 4225, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts are requests. even, bold depos" }
-{ "l_orderkey": 4417, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34933.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "slyly regular, silent courts. even packag" }
-{ "l_orderkey": 5027, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5988.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-28", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-10-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar, ironic deposi" }
-{ "l_orderkey": 5414, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts sleep sl" }
 { "l_orderkey": 1125, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 28944.61, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-29", "l_commitdate": "1994-12-20", "l_receiptdate": "1994-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " platelets wake against the carefully i" }
+{ "l_orderkey": 1159, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-09", "l_commitdate": "1992-12-07", "l_receiptdate": "1992-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "h furiousl" }
+{ "l_orderkey": 1348, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1996.18, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-20", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final packages use fluffily express ac" }
 { "l_orderkey": 1510, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e of the unusual accounts. stealthy deposit" }
+{ "l_orderkey": 1511, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28944.61, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s cajole furiously against " }
 { "l_orderkey": 1888, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26948.43, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-01-16", "l_receiptdate": "1994-02-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". carefully special dolphins sle" }
+{ "l_orderkey": 2561, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4990.45, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p ironic, regular pinto beans." }
+{ "l_orderkey": 2883, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22956.07, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-03", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even requests cajole. special, regular " }
+{ "l_orderkey": 3463, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42917.87, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-28", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " across the " }
 { "l_orderkey": 3526, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-05-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. furiously regular d" }
+{ "l_orderkey": 3617, "l_partkey": 98, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15969.44, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " slyly on th" }
 { "l_orderkey": 3617, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31938.88, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-20", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-05-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uriously against the express accounts. ex" }
 { "l_orderkey": 3777, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13973.26, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-05-31", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ording to the iro" }
+{ "l_orderkey": 3778, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-02", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r deposits. theodol" }
+{ "l_orderkey": 4225, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-11", "l_commitdate": "1997-09-01", "l_receiptdate": "1997-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts are requests. even, bold depos" }
+{ "l_orderkey": 4390, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31938.88, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-15", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ctions across" }
+{ "l_orderkey": 4417, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34933.15, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-09-23", "l_receiptdate": "1998-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "slyly regular, silent courts. even packag" }
+{ "l_orderkey": 4421, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36929.33, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l accounts. ironic request" }
+{ "l_orderkey": 4932, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10978.99, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-21", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-09-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "as. special depende" }
+{ "l_orderkey": 4960, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7984.72, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-14", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-04-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "as. busily regular packages nag. " }
+{ "l_orderkey": 5027, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5988.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-28", "l_commitdate": "1997-11-24", "l_receiptdate": "1997-10-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar, ironic deposi" }
+{ "l_orderkey": 5414, "l_partkey": 98, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 27946.52, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts sleep sl" }
+{ "l_orderkey": 5603, "l_partkey": 98, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49904.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-06", "l_commitdate": "1992-08-20", "l_receiptdate": "1992-10-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "final theodolites accor" }
+{ "l_orderkey": 5696, "l_partkey": 98, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19961.8, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-25", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-07-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "silent, pending ideas sleep fluffil" }
 { "l_orderkey": 71, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32903.97, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-04-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " serve quickly fluffily bold deposi" }
 { "l_orderkey": 226, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3988.36, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-04-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "c foxes integrate carefully against th" }
+{ "l_orderkey": 261, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 19941.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ing to the special, ironic deposi" }
+{ "l_orderkey": 355, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40880.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " deposits. carefully r" }
+{ "l_orderkey": 358, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14956.35, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-12-17", "l_receiptdate": "1993-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "out the blithely ironic deposits slee" }
 { "l_orderkey": 387, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39883.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-04-18", "l_receiptdate": "1997-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " quickly ironic platelets are slyly. fluff" }
+{ "l_orderkey": 484, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9970.9, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "x fluffily carefully regular" }
 { "l_orderkey": 871, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-25", "l_commitdate": "1996-02-09", "l_receiptdate": "1996-03-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "coys dazzle slyly slow notornis. f" }
+{ "l_orderkey": 961, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41877.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ests do cajole blithely. furiously bo" }
 { "l_orderkey": 1057, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-04-30", "l_receiptdate": "1992-06-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y slyly express theodolites. slyly bo" }
 { "l_orderkey": 1154, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4985.45, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furiously " }
+{ "l_orderkey": 1216, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7976.72, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-01", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " of the carefully express" }
 { "l_orderkey": 1606, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19941.8, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-01", "l_commitdate": "1997-05-26", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fily carefu" }
+{ "l_orderkey": 1698, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43871.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-05-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts wake slyly after t" }
+{ "l_orderkey": 1856, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46863.23, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ingly blithe theodolites. slyly pending " }
+{ "l_orderkey": 2048, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes. idly ironic packages nag" }
 { "l_orderkey": 2208, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-07-19", "l_receiptdate": "1995-05-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ding waters lose. furiously regu" }
 { "l_orderkey": 2241, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-07-30", "l_receiptdate": "1993-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss accounts engage furiously. slyly even re" }
+{ "l_orderkey": 2278, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21935.98, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-07-14", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ep regular accounts. blithely even" }
+{ "l_orderkey": 2850, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42874.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1996-11-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "unusual accounts" }
+{ "l_orderkey": 2945, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 44869.05, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1996-03-25", "l_receiptdate": "1996-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ainst the final packages" }
 { "l_orderkey": 3072, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-03-31", "l_receiptdate": "1994-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uests. ironic, ironic depos" }
 { "l_orderkey": 3393, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24927.25, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ng excuses" }
 { "l_orderkey": 3430, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40880.69, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "cuses. silent excuses h" }
-{ "l_orderkey": 5029, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1994.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages. furiously ironi" }
-{ "l_orderkey": 5121, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26921.43, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly silent theodolit" }
-{ "l_orderkey": 261, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 19941.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-09-05", "l_receiptdate": "1993-11-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ing to the special, ironic deposi" }
-{ "l_orderkey": 358, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14956.35, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-12-17", "l_receiptdate": "1993-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "out the blithely ironic deposits slee" }
-{ "l_orderkey": 1698, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43871.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-05-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ts wake slyly after t" }
-{ "l_orderkey": 2278, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21935.98, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-07-14", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ep regular accounts. blithely even" }
-{ "l_orderkey": 2945, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 44869.05, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1996-03-25", "l_receiptdate": "1996-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ainst the final packages" }
+{ "l_orderkey": 3655, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 997.09, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "arefully slow pinto beans are" }
 { "l_orderkey": 3715, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12962.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e quickly ironic" }
 { "l_orderkey": 4035, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3988.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-04-23", "l_receiptdate": "1992-04-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ilent, even pear" }
 { "l_orderkey": 4065, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 32903.97, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ain blithely " }
-{ "l_orderkey": 5987, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 36892.33, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "le furiously carefully special " }
-{ "l_orderkey": 355, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40880.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-15", "l_commitdate": "1994-07-19", "l_receiptdate": "1994-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " deposits. carefully r" }
-{ "l_orderkey": 961, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41877.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ests do cajole blithely. furiously bo" }
-{ "l_orderkey": 1856, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46863.23, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ingly blithe theodolites. slyly pending " }
-{ "l_orderkey": 2048, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "totes. idly ironic packages nag" }
-{ "l_orderkey": 3655, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 997.09, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "arefully slow pinto beans are" }
-{ "l_orderkey": 4800, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic dependenc" }
-{ "l_orderkey": 5286, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-23", "l_commitdate": "1997-12-10", "l_receiptdate": "1997-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y express instructions sleep carefull" }
-{ "l_orderkey": 484, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9970.9, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-04-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "x fluffily carefully regular" }
-{ "l_orderkey": 1216, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7976.72, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-01", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " of the carefully express" }
-{ "l_orderkey": 2850, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42874.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1996-11-03", "l_receiptdate": "1997-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "unusual accounts" }
 { "l_orderkey": 4644, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15953.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-13", "l_commitdate": "1998-02-21", "l_receiptdate": "1998-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lar excuses across the " }
+{ "l_orderkey": 4800, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10967.99, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-27", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic dependenc" }
 { "l_orderkey": 4934, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47860.32, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " ideas cajol" }
+{ "l_orderkey": 5029, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1994.18, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1993-01-04", "l_receiptdate": "1992-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "packages. furiously ironi" }
 { "l_orderkey": 5095, "l_partkey": 97, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14956.35, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-11", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " to the packages wake sly" }
+{ "l_orderkey": 5121, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26921.43, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-06-11", "l_receiptdate": "1992-06-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly silent theodolit" }
 { "l_orderkey": 5280, "l_partkey": 97, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15953.44, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-04-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " foxes are furiously. theodoli" }
+{ "l_orderkey": 5286, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6979.63, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-23", "l_commitdate": "1997-12-10", "l_receiptdate": "1997-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y express instructions sleep carefull" }
 { "l_orderkey": 5412, "l_partkey": 97, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25924.34, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-22", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-02-17", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " the blithel" }
 { "l_orderkey": 5537, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37889.42, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-23", "l_receiptdate": "1996-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s above the carefully ironic deposits " }
+{ "l_orderkey": 5987, "l_partkey": 97, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 36892.33, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-11-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "le furiously carefully special " }
 { "l_orderkey": 260, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 43827.96, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "above the blithely ironic instr" }
-{ "l_orderkey": 1315, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26894.43, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-07-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "latelets. fluffily ironic account" }
-{ "l_orderkey": 2052, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "final requests. stealt" }
-{ "l_orderkey": 3172, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-26", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s are slyly thin package" }
-{ "l_orderkey": 3619, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-22", "l_commitdate": "1996-12-21", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " waters. furiously even deposits " }
-{ "l_orderkey": 4197, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-09-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l instructions print slyly past the reg" }
-{ "l_orderkey": 4225, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-08-31", "l_receiptdate": "1997-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". quickly b" }
-{ "l_orderkey": 4482, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31874.88, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eans wake according " }
-{ "l_orderkey": 4678, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12949.17, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-03", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "its. carefully final fr" }
-{ "l_orderkey": 645, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. slyly iron" }
-{ "l_orderkey": 1159, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6972.63, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-12-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "olve somet" }
-{ "l_orderkey": 1671, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-09-19", "l_receiptdate": "1996-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lyly regular ac" }
-{ "l_orderkey": 3971, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e slyly final dependencies x-ray " }
-{ "l_orderkey": 4067, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13945.26, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ructions. quickly ironic accounts detect " }
-{ "l_orderkey": 4134, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33867.06, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual asymptotes wake carefully alo" }
-{ "l_orderkey": 4229, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43827.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. carefully e" }
-{ "l_orderkey": 5383, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11953.08, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-02", "l_commitdate": "1995-08-16", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y regular instructi" }
-{ "l_orderkey": 5411, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16933.53, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly slyly even deposits. carefully b" }
 { "l_orderkey": 323, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17929.62, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "posits cajole furiously pinto beans. " }
 { "l_orderkey": 453, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 44824.05, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-06-29", "l_receiptdate": "1997-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ironic foxes. slyly pending depos" }
-{ "l_orderkey": 1441, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49804.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " requests. blithely e" }
-{ "l_orderkey": 1760, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37851.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tions. blithely regular orbits against the " }
-{ "l_orderkey": 1920, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23906.16, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-08-23", "l_receiptdate": "1998-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "thely. bold, pend" }
-{ "l_orderkey": 2181, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ward the quietly even requests. ir" }
-{ "l_orderkey": 3943, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-03", "l_receiptdate": "1996-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully ironic " }
-{ "l_orderkey": 4262, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4980.45, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-27", "l_commitdate": "1996-09-05", "l_receiptdate": "1996-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely final asymptotes integrate" }
+{ "l_orderkey": 645, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-02-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. slyly iron" }
 { "l_orderkey": 742, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14941.35, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-26", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-03-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "blithely unusual pinto" }
 { "l_orderkey": 1063, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41835.78, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-10", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tructions about the blithely ex" }
+{ "l_orderkey": 1159, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6972.63, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-12-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "olve somet" }
+{ "l_orderkey": 1315, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26894.43, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-07-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "latelets. fluffily ironic account" }
+{ "l_orderkey": 1441, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49804.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-07", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " requests. blithely e" }
+{ "l_orderkey": 1671, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-09-19", "l_receiptdate": "1996-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lyly regular ac" }
+{ "l_orderkey": 1760, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37851.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-15", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tions. blithely regular orbits against the " }
+{ "l_orderkey": 1920, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23906.16, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-27", "l_commitdate": "1998-08-23", "l_receiptdate": "1998-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "thely. bold, pend" }
+{ "l_orderkey": 2052, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-05-16", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "final requests. stealt" }
+{ "l_orderkey": 2181, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-05", "l_receiptdate": "1996-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ward the quietly even requests. ir" }
+{ "l_orderkey": 3172, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3984.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-26", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s are slyly thin package" }
 { "l_orderkey": 3268, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 996.09, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". ironic, bold requests use carefull" }
 { "l_orderkey": 3590, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42831.87, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s could have to use" }
+{ "l_orderkey": 3619, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48808.41, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-22", "l_commitdate": "1996-12-21", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " waters. furiously even deposits " }
+{ "l_orderkey": 3943, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8964.81, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-03", "l_receiptdate": "1996-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully ironic " }
+{ "l_orderkey": 3971, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e slyly final dependencies x-ray " }
+{ "l_orderkey": 4067, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13945.26, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ructions. quickly ironic accounts detect " }
 { "l_orderkey": 4067, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11953.08, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1992-11-28", "l_receiptdate": "1992-12-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lly slyly even theodol" }
+{ "l_orderkey": 4134, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33867.06, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-05-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ual asymptotes wake carefully alo" }
+{ "l_orderkey": 4197, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-10", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-09-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "l instructions print slyly past the reg" }
+{ "l_orderkey": 4225, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-08-31", "l_receiptdate": "1997-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". quickly b" }
+{ "l_orderkey": 4229, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43827.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. carefully e" }
+{ "l_orderkey": 4262, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4980.45, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-27", "l_commitdate": "1996-09-05", "l_receiptdate": "1996-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely final asymptotes integrate" }
 { "l_orderkey": 4450, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44824.05, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-01", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "express ideas are furiously regular" }
+{ "l_orderkey": 4482, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31874.88, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eans wake according " }
 { "l_orderkey": 4486, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 46816.23, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-05-24", "l_receiptdate": "1998-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ts around the quiet packages ar" }
+{ "l_orderkey": 4678, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12949.17, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-03", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "its. carefully final fr" }
 { "l_orderkey": 5026, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12949.17, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-23", "l_commitdate": "1997-11-02", "l_receiptdate": "1998-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "endencies sleep carefully alongs" }
 { "l_orderkey": 5185, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29882.7, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-17", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ress packages are furiously" }
+{ "l_orderkey": 5383, "l_partkey": 96, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11953.08, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-02", "l_commitdate": "1995-08-16", "l_receiptdate": "1995-08-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y regular instructi" }
+{ "l_orderkey": 5411, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16933.53, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly slyly even deposits. carefully b" }
 { "l_orderkey": 5477, "l_partkey": 96, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 22910.07, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-01-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "telets wake blithely ab" }
 { "l_orderkey": 5541, "l_partkey": 96, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38847.51, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-17", "l_commitdate": "1997-12-27", "l_receiptdate": "1997-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ding theodolites haggle against the slyly " }
 { "l_orderkey": 5637, "l_partkey": 96, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21913.98, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-09-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nding requests are ca" }
 { "l_orderkey": 7, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 45774.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-27", "l_receiptdate": "1996-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual reques" }
-{ "l_orderkey": 68, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19901.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " excuses integrate fluffily " }
-{ "l_orderkey": 1794, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2985.27, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " sentiments according to the q" }
-{ "l_orderkey": 2176, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13931.26, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-07", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely ironic platelets " }
-{ "l_orderkey": 2434, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-02", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " furiously express packages. ironic, pend" }
-{ "l_orderkey": 3328, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41793.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-12-20", "l_receiptdate": "1992-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic requests" }
-{ "l_orderkey": 3430, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4975.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "even accounts haggle slyly bol" }
-{ "l_orderkey": 4327, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17911.62, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-20", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y final excuses. ironic, special requests a" }
-{ "l_orderkey": 4772, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30847.79, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ests are thinly. furiously unusua" }
-{ "l_orderkey": 1348, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12936.17, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely r" }
-{ "l_orderkey": 1831, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ests. express pinto beans abou" }
-{ "l_orderkey": 3590, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18906.71, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "special pinto beans. blithely reg" }
-{ "l_orderkey": 4641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 38808.51, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-10", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the bold reque" }
-{ "l_orderkey": 641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39803.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-10-20", "l_receiptdate": "1993-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lets. furiously regular requests cajo" }
-{ "l_orderkey": 1124, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-07", "l_commitdate": "1998-08-31", "l_receiptdate": "1998-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly bold accou" }
-{ "l_orderkey": 1287, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9950.9, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-08", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely alongside of the unusual, ironic pa" }
-{ "l_orderkey": 3235, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42788.87, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly final instru" }
-{ "l_orderkey": 5799, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40798.69, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-10-31", "l_receiptdate": "1995-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al accounts sleep ruthlessl" }
 { "l_orderkey": 39, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 39803.6, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-22", "l_receiptdate": "1997-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "quickly ironic fox" }
+{ "l_orderkey": 68, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19901.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " excuses integrate fluffily " }
 { "l_orderkey": 453, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 27862.52, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final dependencies. slyly special pl" }
 { "l_orderkey": 610, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4975.45, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-11", "l_commitdate": "1995-10-22", "l_receiptdate": "1995-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "n pinto beans. iro" }
+{ "l_orderkey": 641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39803.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-22", "l_commitdate": "1993-10-20", "l_receiptdate": "1993-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lets. furiously regular requests cajo" }
 { "l_orderkey": 801, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20896.89, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "wake silently furiously idle deposits. " }
 { "l_orderkey": 838, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-26", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ets haggle furiously furiously regular r" }
+{ "l_orderkey": 1124, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-07", "l_commitdate": "1998-08-31", "l_receiptdate": "1998-10-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly bold accou" }
+{ "l_orderkey": 1287, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9950.9, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-08", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thely alongside of the unusual, ironic pa" }
+{ "l_orderkey": 1348, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12936.17, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely r" }
 { "l_orderkey": 1667, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 47764.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-27", "l_commitdate": "1998-01-06", "l_receiptdate": "1998-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "tes sleep furiously. carefully eve" }
+{ "l_orderkey": 1794, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2985.27, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-12-16", "l_receiptdate": "1997-11-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " sentiments according to the q" }
+{ "l_orderkey": 1831, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-21", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ests. express pinto beans abou" }
+{ "l_orderkey": 2176, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13931.26, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1993-01-07", "l_receiptdate": "1992-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely ironic platelets " }
 { "l_orderkey": 2273, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7960.72, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1997-02-27", "l_receiptdate": "1997-01-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "dependencies. slyly ir" }
+{ "l_orderkey": 2434, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 995.09, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-02", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " furiously express packages. ironic, pend" }
 { "l_orderkey": 2628, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22887.07, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-27", "l_commitdate": "1994-01-08", "l_receiptdate": "1993-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usual packages sleep about the fina" }
+{ "l_orderkey": 3235, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42788.87, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-01-23", "l_receiptdate": "1996-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ckly final instru" }
 { "l_orderkey": 3271, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13931.26, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-24", "l_commitdate": "1992-02-14", "l_receiptdate": "1992-03-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending, even packa" }
+{ "l_orderkey": 3328, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41793.78, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-24", "l_commitdate": "1992-12-20", "l_receiptdate": "1992-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic requests" }
+{ "l_orderkey": 3430, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4975.45, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "even accounts haggle slyly bol" }
 { "l_orderkey": 3460, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49754.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "e slyly about the sly" }
 { "l_orderkey": 3461, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 40798.69, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-19", "l_commitdate": "1993-04-20", "l_receiptdate": "1993-02-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heodolites. blithely ironi" }
+{ "l_orderkey": 3590, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18906.71, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "special pinto beans. blithely reg" }
+{ "l_orderkey": 4327, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17911.62, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-20", "l_receiptdate": "1995-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y final excuses. ironic, special requests a" }
 { "l_orderkey": 4484, "l_partkey": 95, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3980.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "packages de" }
+{ "l_orderkey": 4641, "l_partkey": 95, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 38808.51, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-10", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the bold reque" }
+{ "l_orderkey": 4772, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30847.79, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ests are thinly. furiously unusua" }
 { "l_orderkey": 5317, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18906.71, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-12-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "onic requests boost bli" }
+{ "l_orderkey": 5799, "l_partkey": 95, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40798.69, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1995-10-31", "l_receiptdate": "1995-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al accounts sleep ruthlessl" }
+{ "l_orderkey": 193, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22864.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even accounts wake blithely bold" }
 { "l_orderkey": 195, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-03-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rts detect in place of t" }
+{ "l_orderkey": 224, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 44734.05, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "leep furiously regular requests. furiousl" }
 { "l_orderkey": 416, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24852.25, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-11-26", "l_receiptdate": "1993-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y final theodolites about" }
-{ "l_orderkey": 1697, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48710.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1997-01-02", "l_receiptdate": "1996-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar foxes. fluffily furious ideas doubt qu" }
-{ "l_orderkey": 2468, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-16", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "unusual theodolites su" }
-{ "l_orderkey": 3013, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30816.79, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y furious depen" }
-{ "l_orderkey": 4039, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37775.42, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1997-12-31", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "sual asymptotes. ironic deposits nag aft" }
-{ "l_orderkey": 4416, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36781.33, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fluffily ironic " }
+{ "l_orderkey": 1218, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "dolphins. theodolites beyond th" }
 { "l_orderkey": 1281, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1988.18, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1995-01-26", "l_receiptdate": "1995-01-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly unusual requests. final reques" }
 { "l_orderkey": 1696, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 42745.87, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-03-29", "l_receiptdate": "1998-02-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "arefully regular dep" }
-{ "l_orderkey": 3173, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1988.18, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular pearls" }
-{ "l_orderkey": 3650, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 26840.43, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-07-23", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular requests snooze fluffily regular pi" }
-{ "l_orderkey": 3778, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e the furiously ironi" }
-{ "l_orderkey": 4001, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely ironic d" }
-{ "l_orderkey": 4642, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-06-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ily pending accounts hag" }
+{ "l_orderkey": 1697, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48710.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1997-01-02", "l_receiptdate": "1996-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar foxes. fluffily furious ideas doubt qu" }
 { "l_orderkey": 1762, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34793.15, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ind quickly. accounts ca" }
 { "l_orderkey": 2084, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8946.81, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-18", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-03-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "heaves boost slyly after the pla" }
 { "l_orderkey": 2182, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33799.06, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-28", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " slow tithes. ironi" }
+{ "l_orderkey": 2437, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e of the bold, dogged requests" }
+{ "l_orderkey": 2468, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-16", "l_commitdate": "1997-08-09", "l_receiptdate": "1997-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "unusual theodolites su" }
 { "l_orderkey": 2533, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 13917.26, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-06", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ut the pending, special depos" }
 { "l_orderkey": 2915, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11929.08, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-18", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-07-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "accounts. slyly final" }
-{ "l_orderkey": 4353, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21869.98, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ent packages. accounts are slyly. " }
-{ "l_orderkey": 5954, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19881.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-02-05", "l_receiptdate": "1992-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " accounts wake carefu" }
-{ "l_orderkey": 193, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22864.07, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-10-11", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even accounts wake blithely bold" }
-{ "l_orderkey": 224, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 44734.05, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-14", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "leep furiously regular requests. furiousl" }
-{ "l_orderkey": 1218, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "dolphins. theodolites beyond th" }
-{ "l_orderkey": 2437, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 45728.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-12", "l_commitdate": "1993-06-16", "l_receiptdate": "1993-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "e of the bold, dogged requests" }
 { "l_orderkey": 2946, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47716.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-03-31", "l_receiptdate": "1996-06-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oss the platelets. furi" }
+{ "l_orderkey": 3013, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30816.79, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-05", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y furious depen" }
+{ "l_orderkey": 3173, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1988.18, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-15", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular pearls" }
 { "l_orderkey": 3585, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6958.63, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dependencies sleep un" }
+{ "l_orderkey": 3650, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 26840.43, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-07-23", "l_receiptdate": "1992-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ular requests snooze fluffily regular pi" }
 { "l_orderkey": 3654, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33799.06, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-26", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the quick" }
+{ "l_orderkey": 3778, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40757.69, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-21", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e the furiously ironi" }
+{ "l_orderkey": 4001, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely ironic d" }
+{ "l_orderkey": 4039, "l_partkey": 94, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 37775.42, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-09", "l_commitdate": "1997-12-31", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "sual asymptotes. ironic deposits nag aft" }
+{ "l_orderkey": 4353, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21869.98, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-01-23", "l_receiptdate": "1998-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ent packages. accounts are slyly. " }
+{ "l_orderkey": 4416, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36781.33, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-08-23", "l_receiptdate": "1992-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "fluffily ironic " }
+{ "l_orderkey": 4642, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17893.62, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-06-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ily pending accounts hag" }
 { "l_orderkey": 5189, "l_partkey": 94, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48710.41, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-19", "l_receiptdate": "1994-02-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests " }
 { "l_orderkey": 5474, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9940.9, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "pinto bean" }
 { "l_orderkey": 5571, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30816.79, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1993-01-18", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "uffily even accounts. quickly re" }
 { "l_orderkey": 5574, "l_partkey": 94, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13917.26, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-20", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " use slyly carefully special requests? slyl" }
-{ "l_orderkey": 548, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-24", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "courts boost care" }
-{ "l_orderkey": 1508, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42702.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-01", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ndencies h" }
-{ "l_orderkey": 1702, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27806.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-07-26", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts haggle along the packa" }
-{ "l_orderkey": 3138, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6951.63, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-03-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely quickly even packages. packages" }
-{ "l_orderkey": 3493, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30785.79, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ructions. slyly regular accounts across the" }
-{ "l_orderkey": 3619, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17875.62, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites " }
-{ "l_orderkey": 4066, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "quests. slyly regu" }
-{ "l_orderkey": 4166, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es along the furiously regular acc" }
-{ "l_orderkey": 5859, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39723.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-08-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dependenci" }
+{ "l_orderkey": 5954, "l_partkey": 94, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19881.8, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-25", "l_commitdate": "1993-02-05", "l_receiptdate": "1992-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " accounts wake carefu" }
 { "l_orderkey": 69, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-07-26", "l_receiptdate": "1994-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "tect regular, speci" }
 { "l_orderkey": 193, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8937.81, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-08", "l_receiptdate": "1993-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "against the fluffily regular d" }
-{ "l_orderkey": 1124, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34758.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-25", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ut the slyly bold pinto beans; fi" }
-{ "l_orderkey": 1637, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely a" }
-{ "l_orderkey": 2530, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-03-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ng platelets wake s" }
-{ "l_orderkey": 2917, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35751.24, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly ironic d" }
-{ "l_orderkey": 3269, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the special packages. " }
-{ "l_orderkey": 3490, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inal deposits use furiousl" }
-{ "l_orderkey": 3621, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12910.17, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-06-30", "l_receiptdate": "1993-09-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r the unusual packages. brave theodoli" }
-{ "l_orderkey": 3654, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28799.61, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "odolites detect. quickly r" }
-{ "l_orderkey": 4647, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15889.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "o beans about the fluffily special the" }
-{ "l_orderkey": 4928, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19861.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "quiet theodolites ca" }
 { "l_orderkey": 384, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10923.99, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-05-29", "l_receiptdate": "1992-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nic excuses are furiously above the blith" }
-{ "l_orderkey": 1283, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46675.23, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even instructions boost slyly blithely " }
-{ "l_orderkey": 1572, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9930.9, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-05-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " accounts affix slyly. " }
-{ "l_orderkey": 1698, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5958.54, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-21", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending packages affix ne" }
-{ "l_orderkey": 2881, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20854.89, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-07-03", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "hely express Tiresias. final dependencies " }
-{ "l_orderkey": 4487, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24827.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g the final instructions. slyly c" }
+{ "l_orderkey": 548, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-24", "l_commitdate": "1994-11-24", "l_receiptdate": "1994-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "courts boost care" }
 { "l_orderkey": 581, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13903.26, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-17", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": ". deposits s" }
 { "l_orderkey": 640, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48661.41, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-04-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s haggle slyly" }
+{ "l_orderkey": 1124, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34758.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-25", "l_commitdate": "1998-10-08", "l_receiptdate": "1998-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ut the slyly bold pinto beans; fi" }
+{ "l_orderkey": 1283, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46675.23, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-21", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "even instructions boost slyly blithely " }
+{ "l_orderkey": 1508, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42702.87, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-01", "l_commitdate": "1998-06-24", "l_receiptdate": "1998-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ndencies h" }
+{ "l_orderkey": 1572, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9930.9, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-05-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " accounts affix slyly. " }
+{ "l_orderkey": 1637, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-18", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely a" }
+{ "l_orderkey": 1698, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5958.54, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-21", "l_commitdate": "1997-06-08", "l_receiptdate": "1997-09-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " pending packages affix ne" }
+{ "l_orderkey": 1702, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27806.52, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-07-26", "l_receiptdate": "1995-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nts haggle along the packa" }
+{ "l_orderkey": 2530, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-27", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-03-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ng platelets wake s" }
+{ "l_orderkey": 2881, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20854.89, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-07-03", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "hely express Tiresias. final dependencies " }
+{ "l_orderkey": 2917, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35751.24, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-07", "l_commitdate": "1998-02-23", "l_receiptdate": "1998-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly ironic d" }
+{ "l_orderkey": 3138, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6951.63, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-03-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely quickly even packages. packages" }
+{ "l_orderkey": 3269, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41709.78, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " the special packages. " }
+{ "l_orderkey": 3490, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-25", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inal deposits use furiousl" }
+{ "l_orderkey": 3493, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30785.79, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-12", "l_receiptdate": "1993-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ructions. slyly regular accounts across the" }
+{ "l_orderkey": 3619, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 17875.62, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eodolites " }
+{ "l_orderkey": 3621, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12910.17, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-06-30", "l_receiptdate": "1993-09-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "r the unusual packages. brave theodoli" }
+{ "l_orderkey": 3654, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28799.61, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "odolites detect. quickly r" }
+{ "l_orderkey": 4066, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18868.71, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-13", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "quests. slyly regu" }
+{ "l_orderkey": 4166, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7944.72, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "es along the furiously regular acc" }
+{ "l_orderkey": 4487, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24827.25, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g the final instructions. slyly c" }
+{ "l_orderkey": 4647, "l_partkey": 93, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15889.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "o beans about the fluffily special the" }
+{ "l_orderkey": 4928, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19861.8, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1993-11-29", "l_receiptdate": "1994-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "quiet theodolites ca" }
 { "l_orderkey": 5509, "l_partkey": 93, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29792.7, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-23", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "counts haggle pinto beans. furiously " }
 { "l_orderkey": 5857, "l_partkey": 93, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 48661.41, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-12-12", "l_receiptdate": "1998-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "egular pinto beans" }
-{ "l_orderkey": 295, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25794.34, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts above the slyly regular requests x-ray q" }
-{ "l_orderkey": 1186, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s haggle furiously; slyl" }
-{ "l_orderkey": 1474, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-23", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-02-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "after the special" }
-{ "l_orderkey": 3174, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20833.89, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-20", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "iously. idly bold theodolites a" }
-{ "l_orderkey": 3908, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49604.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " even accounts wake " }
-{ "l_orderkey": 5824, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31746.88, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-16", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ven requests. " }
-{ "l_orderkey": 1122, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7936.72, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c foxes are along the slyly r" }
-{ "l_orderkey": 1252, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s are. slyly final requests among the" }
-{ "l_orderkey": 2144, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32738.97, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-04-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic excuses haggle final dependencies. " }
-{ "l_orderkey": 2724, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46628.23, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-13", "l_receiptdate": "1994-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual patterns nag. special p" }
-{ "l_orderkey": 2948, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 48612.41, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ress requests. furiously blithe foxes " }
-{ "l_orderkey": 3490, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-08-06", "l_receiptdate": "1997-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". even requests cajol" }
-{ "l_orderkey": 4580, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nticingly final packag" }
-{ "l_orderkey": 5094, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-24", "l_receiptdate": "1993-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s cajole quickly against the furiously ex" }
-{ "l_orderkey": 5606, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33731.06, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uses. slyly final " }
-{ "l_orderkey": 5987, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-30", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "theodolites wake above the furiously b" }
-{ "l_orderkey": 992, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6944.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-12-24", "l_receiptdate": "1997-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ideas haggle. special theodolit" }
-{ "l_orderkey": 1986, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-06-28", "l_receiptdate": "1994-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sleep furiously fluffily final" }
-{ "l_orderkey": 2658, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " dependencies. blithely pending foxes abou" }
-{ "l_orderkey": 3010, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 37699.42, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-03-16", "l_receiptdate": "1996-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "accounts ar" }
-{ "l_orderkey": 3911, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uctions. blithely regula" }
-{ "l_orderkey": 4615, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9920.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-20", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sits. slyly express deposits are" }
-{ "l_orderkey": 5571, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-11", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "uests haggle furiously pending d" }
+{ "l_orderkey": 5859, "l_partkey": 93, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39723.6, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-08-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dependenci" }
 { "l_orderkey": 164, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25794.34, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s. blithely special courts are blithel" }
+{ "l_orderkey": 295, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25794.34, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-01-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts above the slyly regular requests x-ray q" }
 { "l_orderkey": 487, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46628.23, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-10-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tions. blithely reg" }
+{ "l_orderkey": 992, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6944.63, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-12-24", "l_receiptdate": "1997-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ideas haggle. special theodolit" }
+{ "l_orderkey": 1122, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7936.72, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c foxes are along the slyly r" }
 { "l_orderkey": 1153, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-09", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "special instructions are. unusual, final du" }
+{ "l_orderkey": 1186, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-03", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s haggle furiously; slyl" }
+{ "l_orderkey": 1252, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-09-22", "l_receiptdate": "1997-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s are. slyly final requests among the" }
 { "l_orderkey": 1280, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 18849.71, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-02-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lyly along the furiously regular " }
+{ "l_orderkey": 1474, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-23", "l_commitdate": "1995-03-28", "l_receiptdate": "1995-02-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "after the special" }
+{ "l_orderkey": 1986, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-17", "l_commitdate": "1994-06-28", "l_receiptdate": "1994-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sleep furiously fluffily final" }
+{ "l_orderkey": 2144, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32738.97, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-04", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-04-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ironic excuses haggle final dependencies. " }
+{ "l_orderkey": 2658, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " dependencies. blithely pending foxes abou" }
+{ "l_orderkey": 2724, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46628.23, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-23", "l_commitdate": "1994-11-13", "l_receiptdate": "1994-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unusual patterns nag. special p" }
 { "l_orderkey": 2755, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18849.71, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-11", "l_commitdate": "1992-03-15", "l_receiptdate": "1992-02-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "furiously special deposits" }
+{ "l_orderkey": 2948, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 48612.41, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1994-11-08", "l_receiptdate": "1995-01-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ress requests. furiously blithe foxes " }
+{ "l_orderkey": 3010, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 37699.42, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-03-16", "l_receiptdate": "1996-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "accounts ar" }
+{ "l_orderkey": 3174, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20833.89, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-20", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "iously. idly bold theodolites a" }
+{ "l_orderkey": 3490, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-08-06", "l_receiptdate": "1997-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". even requests cajol" }
 { "l_orderkey": 3776, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 48612.41, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-01-06", "l_receiptdate": "1993-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es: careful warthogs haggle fluffi" }
+{ "l_orderkey": 3908, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49604.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-04-27", "l_receiptdate": "1993-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " even accounts wake " }
+{ "l_orderkey": 3911, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11905.08, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-04", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uctions. blithely regula" }
+{ "l_orderkey": 4580, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21825.98, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "nticingly final packag" }
+{ "l_orderkey": 4615, "l_partkey": 92, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9920.9, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-20", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sits. slyly express deposits are" }
 { "l_orderkey": 4801, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38691.51, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "warhorses wake never for the care" }
+{ "l_orderkey": 5094, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10912.99, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-24", "l_receiptdate": "1993-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s cajole quickly against the furiously ex" }
+{ "l_orderkey": 5571, "l_partkey": 92, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17857.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-11", "l_commitdate": "1993-02-28", "l_receiptdate": "1993-04-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "uests haggle furiously pending d" }
+{ "l_orderkey": 5606, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 33731.06, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-23", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uses. slyly final " }
+{ "l_orderkey": 5824, "l_partkey": 92, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31746.88, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-16", "l_commitdate": "1997-01-24", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ven requests. " }
+{ "l_orderkey": 5987, "l_partkey": 92, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 42659.87, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-30", "l_commitdate": "1996-10-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "theodolites wake above the furiously b" }
 { "l_orderkey": 613, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16848.53, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-23", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-10-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ar dependencie" }
-{ "l_orderkey": 2373, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4955.45, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-06-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily blithely ironic requests" }
-{ "l_orderkey": 2691, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "leep alongside of the accounts. slyly ironi" }
-{ "l_orderkey": 4835, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2973.27, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "etimes final pac" }
-{ "l_orderkey": 5158, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 37661.42, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uffily regular ac" }
-{ "l_orderkey": 2275, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1992-12-10", "l_receiptdate": "1993-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ost across the never express instruction" }
-{ "l_orderkey": 2407, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9910.9, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " pending instructions. theodolites x-" }
-{ "l_orderkey": 2818, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6937.63, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-03-09", "l_receiptdate": "1995-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly according to the r" }
-{ "l_orderkey": 3105, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " detect slyly. blithely unusual requests ar" }
-{ "l_orderkey": 3360, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28741.61, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-02-25", "l_receiptdate": "1998-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "press asymptotes. furiously final " }
-{ "l_orderkey": 3558, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21803.98, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-04-19", "l_receiptdate": "1996-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "refully ironic theodolites are fu" }
-{ "l_orderkey": 5186, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30723.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-09-26", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " accounts use furiously slyly spe" }
 { "l_orderkey": 741, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21803.98, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-07", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ven deposits about the regular, ironi" }
-{ "l_orderkey": 1794, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 36670.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-21", "l_receiptdate": "1998-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. pinto" }
-{ "l_orderkey": 2117, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23786.16, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-27", "l_receiptdate": "1997-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "thely slyly pending platelets. ironic, " }
-{ "l_orderkey": 2181, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14866.35, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-10-27", "l_receiptdate": "1995-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e above the fluffily regul" }
-{ "l_orderkey": 3015, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are slyly carefully special pinto bea" }
-{ "l_orderkey": 3588, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-03", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "special pinto beans cajole slyly. slyly " }
-{ "l_orderkey": 4448, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 40634.69, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pon the permanently even excuses nag " }
-{ "l_orderkey": 4486, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "to the furious, regular foxes play abov" }
-{ "l_orderkey": 5376, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43607.96, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithe packages detect final theodolites. f" }
 { "l_orderkey": 806, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3964.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-09", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eans. quickly ironic ideas " }
 { "l_orderkey": 897, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14866.35, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-25", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r ideas. slyly spec" }
 { "l_orderkey": 961, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17839.62, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-08-14", "l_receiptdate": "1995-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "rmanent foxes haggle speci" }
+{ "l_orderkey": 1794, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 37.0, "l_extendedprice": 36670.33, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-21", "l_receiptdate": "1998-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. pinto" }
+{ "l_orderkey": 2117, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23786.16, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-27", "l_receiptdate": "1997-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "thely slyly pending platelets. ironic, " }
+{ "l_orderkey": 2181, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14866.35, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-10-27", "l_receiptdate": "1995-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e above the fluffily regul" }
+{ "l_orderkey": 2275, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1992-12-10", "l_receiptdate": "1993-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ost across the never express instruction" }
+{ "l_orderkey": 2373, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4955.45, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-03", "l_receiptdate": "1994-06-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily blithely ironic requests" }
+{ "l_orderkey": 2407, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9910.9, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-09-10", "l_receiptdate": "1998-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " pending instructions. theodolites x-" }
 { "l_orderkey": 2503, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26759.43, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-07-24", "l_receiptdate": "1993-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lly even p" }
+{ "l_orderkey": 2691, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10901.99, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "leep alongside of the accounts. slyly ironi" }
+{ "l_orderkey": 2818, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6937.63, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-03-09", "l_receiptdate": "1995-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly according to the r" }
+{ "l_orderkey": 3015, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are slyly carefully special pinto bea" }
 { "l_orderkey": 3043, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4955.45, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-07-02", "l_receiptdate": "1992-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ake blithely re" }
 { "l_orderkey": 3077, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39643.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-22", "l_commitdate": "1997-09-19", "l_receiptdate": "1997-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "to the enticing packag" }
+{ "l_orderkey": 3105, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22795.07, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1996-12-14", "l_receiptdate": "1997-03-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " detect slyly. blithely unusual requests ar" }
+{ "l_orderkey": 3360, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28741.61, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-02-25", "l_receiptdate": "1998-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "press asymptotes. furiously final " }
+{ "l_orderkey": 3558, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21803.98, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-04-19", "l_receiptdate": "1996-04-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "refully ironic theodolites are fu" }
+{ "l_orderkey": 3588, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-03", "l_commitdate": "1995-05-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "special pinto beans cajole slyly. slyly " }
 { "l_orderkey": 4321, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3964.36, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-10", "l_commitdate": "1994-10-06", "l_receiptdate": "1994-09-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic deposi" }
+{ "l_orderkey": 4448, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 40634.69, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-30", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pon the permanently even excuses nag " }
+{ "l_orderkey": 4486, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27750.52, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "to the furious, regular foxes play abov" }
+{ "l_orderkey": 4835, "l_partkey": 91, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2973.27, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1995-01-12", "l_receiptdate": "1995-02-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "etimes final pac" }
+{ "l_orderkey": 5158, "l_partkey": 91, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 37661.42, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-09", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uffily regular ac" }
+{ "l_orderkey": 5186, "l_partkey": 91, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30723.79, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-19", "l_commitdate": "1996-09-26", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " accounts use furiously slyly spe" }
+{ "l_orderkey": 5376, "l_partkey": 91, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43607.96, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-09-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithe packages detect final theodolites. f" }
 { "l_orderkey": 34, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21781.98, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-10-16", "l_receiptdate": "1998-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "thely slyly p" }
 { "l_orderkey": 133, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10890.99, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-01-15", "l_receiptdate": "1998-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e quickly across the dolphins" }
-{ "l_orderkey": 2400, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 990.09, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-18", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "silent deposits serve furious" }
-{ "l_orderkey": 3559, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "l, regular accounts wake flu" }
-{ "l_orderkey": 3713, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-08-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "totes. carefully special theodolites s" }
-{ "l_orderkey": 4576, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4950.45, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express, special asymptote" }
-{ "l_orderkey": 5186, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y regular notornis k" }
-{ "l_orderkey": 5670, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26732.43, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " ideas promise bli" }
-{ "l_orderkey": 1509, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 36633.33, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he slyly even deposits wake a" }
-{ "l_orderkey": 2272, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons along the blithely e" }
-{ "l_orderkey": 2311, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41583.78, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-06-27", "l_receiptdate": "1995-06-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gle furiously. bold " }
-{ "l_orderkey": 2593, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27722.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y even escapades shall" }
-{ "l_orderkey": 2917, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3960.36, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. unusual instruct" }
-{ "l_orderkey": 2944, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-30", "l_commitdate": "1997-11-03", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fluffily blithely express pea" }
-{ "l_orderkey": 3105, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-28", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. blithely unusual ideas was after" }
-{ "l_orderkey": 3143, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21781.98, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-03-26", "l_receiptdate": "1993-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, special instructions nag " }
-{ "l_orderkey": 3170, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 31682.88, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggle about the furiously r" }
-{ "l_orderkey": 3552, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns after the blithely reg" }
-{ "l_orderkey": 3719, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12871.17, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously. regular dep" }
-{ "l_orderkey": 4386, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-04-09", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " deposits use according to the pending, " }
-{ "l_orderkey": 5056, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22772.07, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ickly regular requests cajole. depos" }
-{ "l_orderkey": 5536, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13861.26, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "instructions sleep " }
-{ "l_orderkey": 5669, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-05", "l_commitdate": "1996-06-10", "l_receiptdate": "1996-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "to beans against the regular depo" }
-{ "l_orderkey": 5922, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34653.15, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-21", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. regu" }
-{ "l_orderkey": 5926, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-07-20", "l_receiptdate": "1994-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle furiously express foxes. bo" }
-{ "l_orderkey": 710, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24752.25, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eas detect do" }
-{ "l_orderkey": 1058, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously f" }
-{ "l_orderkey": 2592, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully special theodolites integrate " }
-{ "l_orderkey": 3364, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48514.41, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-17", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "d accounts? caref" }
-{ "l_orderkey": 4067, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39603.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar theodolites nag blithely above the" }
-{ "l_orderkey": 5445, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32672.97, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-10-14", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ests. final instructions" }
-{ "l_orderkey": 5474, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-06-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nstructions. furio" }
-{ "l_orderkey": 5636, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20791.89, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-05-11", "l_receiptdate": "1995-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " are furiously unusual " }
-{ "l_orderkey": 5986, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-09-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "structions! furiously pending instructi" }
 { "l_orderkey": 359, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37623.42, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g furiously. regular, sile" }
 { "l_orderkey": 646, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 16831.53, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1994-12-26", "l_receiptdate": "1995-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inal packages haggle carefully " }
+{ "l_orderkey": 710, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24752.25, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "eas detect do" }
+{ "l_orderkey": 1058, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-06-21", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uriously f" }
+{ "l_orderkey": 1509, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 36633.33, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-31", "l_commitdate": "1993-09-10", "l_receiptdate": "1993-09-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he slyly even deposits wake a" }
 { "l_orderkey": 1568, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35643.24, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-04-22", "l_receiptdate": "1997-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "platelets-- furiously sly excu" }
 { "l_orderkey": 1601, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13861.26, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-11-22", "l_receiptdate": "1994-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "he special, fin" }
 { "l_orderkey": 1635, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-04-21", "l_receiptdate": "1997-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ravely carefully express " }
 { "l_orderkey": 1827, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 46534.23, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-01", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". pending courts about the even e" }
+{ "l_orderkey": 2272, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ons along the blithely e" }
+{ "l_orderkey": 2311, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41583.78, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-06-27", "l_receiptdate": "1995-06-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gle furiously. bold " }
+{ "l_orderkey": 2400, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 990.09, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-18", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-09-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "silent deposits serve furious" }
 { "l_orderkey": 2563, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49504.5, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly regular, regular excuses. bold plate" }
+{ "l_orderkey": 2592, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-13", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully special theodolites integrate " }
+{ "l_orderkey": 2593, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27722.52, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y even escapades shall" }
 { "l_orderkey": 2628, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 49504.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-13", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-01-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "posits serve carefully toward " }
+{ "l_orderkey": 2917, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3960.36, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s. unusual instruct" }
+{ "l_orderkey": 2944, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 6930.63, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-30", "l_commitdate": "1997-11-03", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fluffily blithely express pea" }
 { "l_orderkey": 2978, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-03", "l_commitdate": "1995-07-25", "l_receiptdate": "1995-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ecial ideas promise slyly" }
+{ "l_orderkey": 3105, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-28", "l_commitdate": "1996-12-28", "l_receiptdate": "1997-01-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. blithely unusual ideas was after" }
+{ "l_orderkey": 3143, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21781.98, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-03-26", "l_receiptdate": "1993-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, special instructions nag " }
+{ "l_orderkey": 3170, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 31682.88, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ggle about the furiously r" }
 { "l_orderkey": 3335, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46534.23, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-12-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " quickly special ideas." }
+{ "l_orderkey": 3364, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48514.41, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-17", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "d accounts? caref" }
 { "l_orderkey": 3461, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15841.44, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-04-29", "l_receiptdate": "1993-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " pending deposi" }
+{ "l_orderkey": 3552, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ns after the blithely reg" }
+{ "l_orderkey": 3559, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-10", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "l, regular accounts wake flu" }
+{ "l_orderkey": 3713, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-08-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "totes. carefully special theodolites s" }
+{ "l_orderkey": 3719, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12871.17, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously. regular dep" }
+{ "l_orderkey": 4067, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39603.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar theodolites nag blithely above the" }
 { "l_orderkey": 4162, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28712.61, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-25", "l_commitdate": "1992-04-25", "l_receiptdate": "1992-03-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nding pinto beans haggle blithe" }
+{ "l_orderkey": 4386, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17821.62, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-04-09", "l_receiptdate": "1998-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " deposits use according to the pending, " }
+{ "l_orderkey": 4576, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4950.45, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-11-08", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly express, special asymptote" }
+{ "l_orderkey": 5056, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22772.07, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-12", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-05-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ickly regular requests cajole. depos" }
+{ "l_orderkey": 5186, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y regular notornis k" }
+{ "l_orderkey": 5445, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32672.97, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-10-14", "l_receiptdate": "1993-10-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ests. final instructions" }
+{ "l_orderkey": 5474, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 45544.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-07", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-06-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nstructions. furio" }
+{ "l_orderkey": 5536, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13861.26, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "instructions sleep " }
+{ "l_orderkey": 5636, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20791.89, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-13", "l_commitdate": "1995-05-11", "l_receiptdate": "1995-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " are furiously unusual " }
+{ "l_orderkey": 5669, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-05", "l_commitdate": "1996-06-10", "l_receiptdate": "1996-08-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "to beans against the regular depo" }
+{ "l_orderkey": 5670, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26732.43, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-09", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " ideas promise bli" }
 { "l_orderkey": 5829, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 41583.78, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-26", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pearls. slyly bold deposits solve final" }
+{ "l_orderkey": 5922, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34653.15, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1997-01-21", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "accounts. regu" }
+{ "l_orderkey": 5926, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7920.72, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-17", "l_commitdate": "1994-07-20", "l_receiptdate": "1994-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle furiously express foxes. bo" }
 { "l_orderkey": 5927, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43563.96, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1997-11-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rding to the special, final decoy" }
+{ "l_orderkey": 5986, "l_partkey": 90, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 30692.79, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-21", "l_commitdate": "1992-06-29", "l_receiptdate": "1992-09-14", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "structions! furiously pending instructi" }
 { "l_orderkey": 4, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "- quickly regular packages sleep. idly" }
-{ "l_orderkey": 1702, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33628.72, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-04", "l_commitdate": "1995-06-08", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y careful packages; dogged acco" }
-{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10879.88, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-31", "l_commitdate": "1992-10-04", "l_receiptdate": "1992-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits promise carefully even, regular e" }
-{ "l_orderkey": 5121, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45497.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "use express foxes. slyly " }
-{ "l_orderkey": 5186, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "capades. accounts sublate. pinto" }
-{ "l_orderkey": 5568, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34617.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lyly. blit" }
-{ "l_orderkey": 5762, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al instructions. furiousl" }
-{ "l_orderkey": 2081, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "among the slyly express accounts. silen" }
-{ "l_orderkey": 2405, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17803.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "carefully ironic accounts. slyly " }
-{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 37585.04, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly final requests. regu" }
-{ "l_orderkey": 4705, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-04-28", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "blithely. sly" }
-{ "l_orderkey": 5190, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44508.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y carefully final ideas. f" }
-{ "l_orderkey": 256, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21759.76, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1993-12-28", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke quickly ironic, ironic deposits. reg" }
-{ "l_orderkey": 2688, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-18", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ithely final " }
-{ "l_orderkey": 3110, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-01-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c theodolites a" }
-{ "l_orderkey": 3170, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26705.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "efully bold foxes. regular, ev" }
-{ "l_orderkey": 3429, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27694.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nstructions boost. thin" }
-{ "l_orderkey": 3649, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22748.84, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-10-01", "l_receiptdate": "1994-09-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rs promise blithe" }
-{ "l_orderkey": 4549, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " requests wake. furiously even " }
 { "l_orderkey": 34, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12858.04, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-23", "l_commitdate": "1998-09-14", "l_receiptdate": "1998-11-06", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nic accounts. deposits are alon" }
 { "l_orderkey": 102, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36595.96, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-24", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ully across the ideas. final deposit" }
+{ "l_orderkey": 256, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21759.76, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1993-12-28", "l_receiptdate": "1994-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke quickly ironic, ironic deposits. reg" }
 { "l_orderkey": 1058, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4945.4, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-05-29", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "refully even requests boost along" }
+{ "l_orderkey": 1702, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33628.72, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-04", "l_commitdate": "1995-06-08", "l_receiptdate": "1995-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y careful packages; dogged acco" }
+{ "l_orderkey": 2081, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-21", "l_commitdate": "1997-10-03", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "among the slyly express accounts. silen" }
+{ "l_orderkey": 2405, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17803.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "carefully ironic accounts. slyly " }
+{ "l_orderkey": 2688, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-18", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ithely final " }
 { "l_orderkey": 2753, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29672.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-01-29", "l_receiptdate": "1994-02-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ans wake fluffily blithely iro" }
+{ "l_orderkey": 3110, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-15", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-01-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c theodolites a" }
 { "l_orderkey": 3140, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9890.8, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-30", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "accounts. expres" }
+{ "l_orderkey": 3170, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26705.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "efully bold foxes. regular, ev" }
+{ "l_orderkey": 3429, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27694.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nstructions boost. thin" }
 { "l_orderkey": 3622, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3956.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-02-19", "l_receiptdate": "1996-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lithely brave foxes. furi" }
+{ "l_orderkey": 3649, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22748.84, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-10-01", "l_receiptdate": "1994-09-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rs promise blithe" }
 { "l_orderkey": 3940, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7912.64, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-04", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ions cajole furiously regular pinto beans. " }
+{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 37585.04, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly final requests. regu" }
+{ "l_orderkey": 4384, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10879.88, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-31", "l_commitdate": "1992-10-04", "l_receiptdate": "1992-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "deposits promise carefully even, regular e" }
 { "l_orderkey": 4416, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2967.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-11-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " requests sleep along the " }
+{ "l_orderkey": 4549, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 989.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-04", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " requests wake. furiously even " }
+{ "l_orderkey": 4705, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-04-28", "l_receiptdate": "1992-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "blithely. sly" }
+{ "l_orderkey": 5121, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45497.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-07-19", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "use express foxes. slyly " }
+{ "l_orderkey": 5186, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25716.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "capades. accounts sublate. pinto" }
+{ "l_orderkey": 5190, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44508.6, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y carefully final ideas. f" }
+{ "l_orderkey": 5568, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34617.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-17", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-10-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lyly. blit" }
+{ "l_orderkey": 5762, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39563.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-09", "l_receiptdate": "1997-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al instructions. furiousl" }
 { "l_orderkey": 5925, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49454.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-14", "l_commitdate": "1996-01-10", "l_receiptdate": "1996-02-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. stealthily express pains print bli" }
 { "l_orderkey": 67, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 43475.52, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-04-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "se quickly above the even, express reques" }
-{ "l_orderkey": 2181, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45451.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "osits. final packages sleep" }
-{ "l_orderkey": 3394, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25690.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "its use furiously. even, even account" }
-{ "l_orderkey": 3623, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-04", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic somas sleep fluffily" }
-{ "l_orderkey": 5158, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely fina" }
-{ "l_orderkey": 5184, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-27", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es above the care" }
+{ "l_orderkey": 99, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9880.8, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. requ" }
 { "l_orderkey": 483, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " carefully express ins" }
-{ "l_orderkey": 1059, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44463.6, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "riously even theodolites. slyly regula" }
-{ "l_orderkey": 2595, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-03-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ctions. regula" }
-{ "l_orderkey": 3045, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40511.28, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-11-24", "l_receiptdate": "1995-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final foxes. carefully ironic pinto b" }
-{ "l_orderkey": 3588, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5928.48, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. fluffily fluf" }
-{ "l_orderkey": 3842, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14821.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ave packages are slyl" }
+{ "l_orderkey": 612, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 988.08, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " requests." }
 { "l_orderkey": 837, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23713.92, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-27", "l_commitdate": "1994-09-02", "l_receiptdate": "1994-07-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p carefully. theodolites use. bold courts a" }
 { "l_orderkey": 1031, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6916.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-11-11", "l_receiptdate": "1994-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "r instructions. car" }
+{ "l_orderkey": 1059, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 44463.6, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-05-08", "l_receiptdate": "1994-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "riously even theodolites. slyly regula" }
 { "l_orderkey": 1792, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1993-12-11", "l_receiptdate": "1994-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "final packages s" }
+{ "l_orderkey": 2181, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45451.68, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "osits. final packages sleep" }
 { "l_orderkey": 2375, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41499.36, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-02-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "apades. idea" }
-{ "l_orderkey": 2854, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28654.32, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y slyly ironic accounts. foxes haggle slyl" }
-{ "l_orderkey": 3878, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12845.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-06-03", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep ruthlessly about the carefu" }
-{ "l_orderkey": 4293, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24702.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "inal asympt" }
-{ "l_orderkey": 4640, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " accounts. unu" }
-{ "l_orderkey": 5703, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1976.16, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-07-26", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nts against the blithely sile" }
-{ "l_orderkey": 99, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9880.8, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-03", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. requ" }
-{ "l_orderkey": 612, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 988.08, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " requests." }
 { "l_orderkey": 2469, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 34582.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-02-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ld packages haggle regular frets. fluffily " }
+{ "l_orderkey": 2595, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-03-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ctions. regula" }
+{ "l_orderkey": 2854, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28654.32, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-06", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y slyly ironic accounts. foxes haggle slyl" }
+{ "l_orderkey": 3045, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 40511.28, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-30", "l_commitdate": "1995-11-24", "l_receiptdate": "1995-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ely final foxes. carefully ironic pinto b" }
 { "l_orderkey": 3072, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 988.08, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-03-14", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " slyly ironic attainments. car" }
+{ "l_orderkey": 3394, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25690.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-08", "l_commitdate": "1996-06-12", "l_receiptdate": "1996-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "its use furiously. even, even account" }
+{ "l_orderkey": 3588, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5928.48, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "s. fluffily fluf" }
+{ "l_orderkey": 3623, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29642.4, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-04", "l_commitdate": "1997-03-03", "l_receiptdate": "1997-05-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic somas sleep fluffily" }
+{ "l_orderkey": 3842, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14821.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ave packages are slyl" }
+{ "l_orderkey": 3878, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12845.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-06-03", "l_receiptdate": "1997-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep ruthlessly about the carefu" }
 { "l_orderkey": 3970, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1976.16, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-24", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "carefully pending foxes wake blithely " }
+{ "l_orderkey": 4293, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24702.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "inal asympt" }
 { "l_orderkey": 4640, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4940.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " warthogs against the regular" }
+{ "l_orderkey": 4640, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8892.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " accounts. unu" }
+{ "l_orderkey": 5158, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely fina" }
+{ "l_orderkey": 5184, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38535.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-27", "l_commitdate": "1998-10-17", "l_receiptdate": "1998-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es above the care" }
 { "l_orderkey": 5442, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 44463.6, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-02-24", "l_receiptdate": "1998-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "old slyly after " }
+{ "l_orderkey": 5703, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1976.16, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-07-26", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nts against the blithely sile" }
 { "l_orderkey": 5957, "l_partkey": 88, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39523.2, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ironic asymptotes sleep blithely again" }
-{ "l_orderkey": 1540, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "carefully final packages; b" }
-{ "l_orderkey": 3556, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27638.24, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully final instructions? ironic packa" }
-{ "l_orderkey": 4196, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 42444.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es. slyly even " }
-{ "l_orderkey": 4294, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully; furiously ex" }
-{ "l_orderkey": 4610, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20728.68, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly special theodolites. even," }
-{ "l_orderkey": 4772, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 987.08, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ans. slyly even acc" }
-{ "l_orderkey": 4932, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4935.4, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-09-13", "l_receiptdate": "1993-10-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " haggle furiously. slyly ironic packages sl" }
-{ "l_orderkey": 5252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-17", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "bold requests. furious" }
-{ "l_orderkey": 5572, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18754.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "es. final, final requests wake blithely ag" }
 { "l_orderkey": 160, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21715.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-18", "l_commitdate": "1997-03-05", "l_receiptdate": "1997-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ncies about the request" }
-{ "l_orderkey": 1156, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14806.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-21", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "the furiously pen" }
-{ "l_orderkey": 1252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12832.04, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sts dazzle" }
-{ "l_orderkey": 2433, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38496.12, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final asy" }
-{ "l_orderkey": 3522, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ve the quickly special packages" }
-{ "l_orderkey": 3558, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7896.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-31", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "? even requests sle" }
-{ "l_orderkey": 4644, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9870.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the slow, final fo" }
-{ "l_orderkey": 5925, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-01-13", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the furiously" }
 { "l_orderkey": 451, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 987.08, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-13", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully ironic packages solve furiously " }
-{ "l_orderkey": 2882, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46392.76, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-13", "l_commitdate": "1995-09-21", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l, special" }
-{ "l_orderkey": 3107, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously final " }
-{ "l_orderkey": 4132, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17767.44, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-01", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y final de" }
-{ "l_orderkey": 4167, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16780.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly around the even instr" }
-{ "l_orderkey": 5281, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-31", "l_commitdate": "1995-12-23", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss the furiously " }
 { "l_orderkey": 576, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1974.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-05-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ccounts along the ac" }
 { "l_orderkey": 709, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6909.56, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-14", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " special orbits cajole " }
 { "l_orderkey": 865, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14806.2, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-07-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " deposits sleep quickl" }
 { "l_orderkey": 1093, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6909.56, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-24", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-11-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "bold deposits. blithely ironic depos" }
+{ "l_orderkey": 1156, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14806.2, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-21", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "the furiously pen" }
+{ "l_orderkey": 1252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12832.04, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-09-12", "l_receiptdate": "1997-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "sts dazzle" }
+{ "l_orderkey": 1540, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-10-18", "l_receiptdate": "1992-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "carefully final packages; b" }
 { "l_orderkey": 1766, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31586.56, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1996-11-11", "l_receiptdate": "1997-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ess accounts. stealthily ironic accou" }
+{ "l_orderkey": 2433, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38496.12, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ly final asy" }
+{ "l_orderkey": 2882, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46392.76, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-13", "l_commitdate": "1995-09-21", "l_receiptdate": "1995-09-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l, special" }
 { "l_orderkey": 3073, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9870.8, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " furiously caref" }
+{ "l_orderkey": 3107, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26651.16, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously final " }
+{ "l_orderkey": 3522, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-05", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ve the quickly special packages" }
+{ "l_orderkey": 3556, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27638.24, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "refully final instructions? ironic packa" }
+{ "l_orderkey": 3558, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7896.64, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-31", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "? even requests sle" }
+{ "l_orderkey": 4132, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17767.44, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-01", "l_commitdate": "1995-08-01", "l_receiptdate": "1995-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y final de" }
+{ "l_orderkey": 4167, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16780.36, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly around the even instr" }
+{ "l_orderkey": 4196, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 42444.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-12", "l_receiptdate": "1998-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es. slyly even " }
+{ "l_orderkey": 4294, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully; furiously ex" }
 { "l_orderkey": 4545, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8883.72, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-02-23", "l_receiptdate": "1993-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "xpress accounts" }
 { "l_orderkey": 4583, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 31586.56, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "across the pinto beans-- quickly" }
+{ "l_orderkey": 4610, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20728.68, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-08-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly special theodolites. even," }
+{ "l_orderkey": 4644, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9870.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-12", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the slow, final fo" }
+{ "l_orderkey": 4772, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 987.08, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ans. slyly even acc" }
+{ "l_orderkey": 4932, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4935.4, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-01", "l_commitdate": "1993-09-13", "l_receiptdate": "1993-10-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " haggle furiously. slyly ironic packages sl" }
 { "l_orderkey": 5027, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 24677.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-16", "l_commitdate": "1997-11-25", "l_receiptdate": "1997-10-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ic ideas. requests sleep fluffily am" }
 { "l_orderkey": 5056, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13819.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-09", "l_commitdate": "1997-04-13", "l_receiptdate": "1997-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts haggle carefully along the slyl" }
+{ "l_orderkey": 5252, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-17", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "bold requests. furious" }
+{ "l_orderkey": 5281, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 47379.84, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-31", "l_commitdate": "1995-12-23", "l_receiptdate": "1996-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss the furiously " }
+{ "l_orderkey": 5572, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18754.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-12", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "es. final, final requests wake blithely ag" }
 { "l_orderkey": 5605, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49354.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "instructions sleep carefully ironic req" }
+{ "l_orderkey": 5925, "l_partkey": 87, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41457.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-01-13", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the furiously" }
+{ "l_orderkey": 32, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 43387.52, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "symptotes nag according to the ironic depo" }
 { "l_orderkey": 35, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-26", "l_commitdate": "1995-12-25", "l_receiptdate": "1995-12-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly unti" }
 { "l_orderkey": 64, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20707.68, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ch slyly final, thin platelets." }
-{ "l_orderkey": 1153, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14791.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-07-17", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uctions boost fluffily according to" }
-{ "l_orderkey": 1570, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6902.56, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-01", "l_receiptdate": "1998-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "requests boost quickly re" }
-{ "l_orderkey": 2371, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38457.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "tructions. regular, stealthy packages wak" }
-{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44373.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-11-27", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "furiously special idea" }
-{ "l_orderkey": 3111, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4930.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-10-16", "l_receiptdate": "1995-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully even ideas" }
-{ "l_orderkey": 3490, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49304.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-06-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " haggle carefu" }
-{ "l_orderkey": 4354, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake slyly eve" }
-{ "l_orderkey": 4999, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-21", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole among the blithel" }
-{ "l_orderkey": 5985, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3944.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-04", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ole along the quickly slow d" }
 { "l_orderkey": 195, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33526.72, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-02-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " cajole furiously bold i" }
 { "l_orderkey": 195, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 40429.28, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-03-13", "l_receiptdate": "1994-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ggle fluffily foxes. fluffily ironic ex" }
-{ "l_orderkey": 1092, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1972.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ans. slyly eve" }
-{ "l_orderkey": 1539, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10846.88, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-27", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly express requests. furiously " }
-{ "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" }
-{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11832.96, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-22", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the slyly ironic dolphins; fin" }
-{ "l_orderkey": 4835, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26624.16, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-13", "l_receiptdate": "1995-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts after the car" }
-{ "l_orderkey": 32, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 44.0, "l_extendedprice": 43387.52, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "symptotes nag according to the ironic depo" }
 { "l_orderkey": 772, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 34512.8, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-06-13", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng ideas. special packages haggle alon" }
+{ "l_orderkey": 1092, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1972.16, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-05-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ans. slyly eve" }
+{ "l_orderkey": 1153, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14791.2, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-07-17", "l_receiptdate": "1996-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "uctions boost fluffily according to" }
 { "l_orderkey": 1507, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38457.12, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-04", "l_commitdate": "1993-12-16", "l_receiptdate": "1993-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ly even instructions." }
+{ "l_orderkey": 1539, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10846.88, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-27", "l_commitdate": "1995-04-13", "l_receiptdate": "1995-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly express requests. furiously " }
+{ "l_orderkey": 1570, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6902.56, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-01", "l_receiptdate": "1998-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "requests boost quickly re" }
+{ "l_orderkey": 1637, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 48317.92, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". blithely i" }
 { "l_orderkey": 1988, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8874.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1996-01-02", "l_receiptdate": "1996-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lar platelets. slyly ironic packa" }
 { "l_orderkey": 2240, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9860.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "are across the ironic packages." }
 { "l_orderkey": 2245, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 32540.64, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-26", "l_commitdate": "1993-06-11", "l_receiptdate": "1993-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ing to the carefully ruthless accounts" }
+{ "l_orderkey": 2371, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38457.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-01", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "tructions. regular, stealthy packages wak" }
 { "l_orderkey": 2402, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42401.44, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-11-20", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly slyly blithe sheaves" }
 { "l_orderkey": 2531, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19721.6, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "into beans. furious" }
+{ "l_orderkey": 2595, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ". final orbits cajole " }
 { "l_orderkey": 2690, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-20", "l_commitdate": "1996-06-01", "l_receiptdate": "1996-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "d accounts above the express req" }
 { "l_orderkey": 2695, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39443.2, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-02", "l_commitdate": "1996-10-26", "l_receiptdate": "1996-11-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ructions. pending" }
+{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44373.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-11-27", "l_receiptdate": "1996-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "furiously special idea" }
+{ "l_orderkey": 2823, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11832.96, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-22", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the slyly ironic dolphins; fin" }
 { "l_orderkey": 3106, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21693.76, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "structions atop the blithely" }
+{ "l_orderkey": 3111, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4930.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-30", "l_commitdate": "1995-10-16", "l_receiptdate": "1995-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully even ideas" }
 { "l_orderkey": 3397, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32540.64, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular accounts. blithely re" }
+{ "l_orderkey": 3490, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49304.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-06-28", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " haggle carefu" }
+{ "l_orderkey": 4166, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-01", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "unts. furiously express accounts w" }
+{ "l_orderkey": 4354, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-20", "l_commitdate": "1994-12-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake slyly eve" }
+{ "l_orderkey": 4835, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26624.16, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-10", "l_commitdate": "1994-12-13", "l_receiptdate": "1995-01-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts after the car" }
 { "l_orderkey": 4896, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 20707.68, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-11-18", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly express deposits. carefully pending depo" }
+{ "l_orderkey": 4999, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-21", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole among the blithel" }
+{ "l_orderkey": 5088, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-05-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously final deposits. furiously re" }
 { "l_orderkey": 5702, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36484.96, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1993-10-21", "l_receiptdate": "1994-01-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ix slyly. regular instructions slee" }
 { "l_orderkey": 5861, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5916.48, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-05-18", "l_receiptdate": "1997-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "olites. slyly" }
-{ "l_orderkey": 2595, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29582.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ". final orbits cajole " }
-{ "l_orderkey": 4166, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-01", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "unts. furiously express accounts w" }
-{ "l_orderkey": 5088, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35498.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-05-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously final deposits. furiously re" }
-{ "l_orderkey": 739, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27582.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets about the pe" }
-{ "l_orderkey": 800, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-10-01", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ckly even requests after the carefully r" }
-{ "l_orderkey": 1057, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-28", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-03-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ar orbits boost bli" }
-{ "l_orderkey": 1221, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-27", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress accounts " }
-{ "l_orderkey": 2021, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts boost blithely. blithely reg" }
-{ "l_orderkey": 3076, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43343.52, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-14", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " instructions h" }
-{ "l_orderkey": 3783, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49254.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he furiously regular deposits. " }
-{ "l_orderkey": 4067, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 16746.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r accounts. slyly special pa" }
+{ "l_orderkey": 5985, "l_partkey": 86, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3944.32, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-04", "l_commitdate": "1995-04-01", "l_receiptdate": "1995-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ole along the quickly slow d" }
 { "l_orderkey": 195, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5910.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-09", "l_commitdate": "1994-03-27", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y, even deposits haggle carefully. bli" }
-{ "l_orderkey": 326, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4925.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas sleep according to the sometimes spe" }
-{ "l_orderkey": 546, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15761.28, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1996-12-30", "l_receiptdate": "1997-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "de of the orbits. sometimes regula" }
-{ "l_orderkey": 1636, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1970.16, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-10-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nal foxes cajole above the blithely reg" }
-{ "l_orderkey": 2081, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-06", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ual requests wake blithely above the" }
-{ "l_orderkey": 2211, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-05", "l_commitdate": "1994-09-13", "l_receiptdate": "1994-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ependencies " }
-{ "l_orderkey": 3175, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13791.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-21", "l_commitdate": "1994-09-05", "l_receiptdate": "1994-11-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nt dependencies are quietly even " }
-{ "l_orderkey": 5574, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18716.52, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old deposits int" }
 { "l_orderkey": 263, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8865.72, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-07-16", "l_receiptdate": "1994-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lms wake bl" }
+{ "l_orderkey": 326, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4925.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-08-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas sleep according to the sometimes spe" }
+{ "l_orderkey": 390, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 23641.92, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-18", "l_commitdate": "1998-05-19", "l_receiptdate": "1998-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. enticingly final depos" }
+{ "l_orderkey": 546, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15761.28, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1996-12-30", "l_receiptdate": "1997-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "de of the orbits. sometimes regula" }
 { "l_orderkey": 580, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32507.64, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-11", "l_commitdate": "1997-09-19", "l_receiptdate": "1997-10-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y express theodolites cajole carefully " }
 { "l_orderkey": 644, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 32507.64, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-26", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-08-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ages sleep. bold, bo" }
+{ "l_orderkey": 739, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27582.24, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets about the pe" }
+{ "l_orderkey": 800, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-10-01", "l_receiptdate": "1998-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ckly even requests after the carefully r" }
 { "l_orderkey": 899, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3940.32, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-06-28", "l_receiptdate": "1998-06-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ter the carefully regular deposits are agai" }
 { "l_orderkey": 967, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3940.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-07-27", "l_receiptdate": "1992-07-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "platelets hang carefully along " }
+{ "l_orderkey": 1057, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20686.68, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-28", "l_commitdate": "1992-05-01", "l_receiptdate": "1992-03-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ar orbits boost bli" }
+{ "l_orderkey": 1221, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-27", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-07-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress accounts " }
+{ "l_orderkey": 1636, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1970.16, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-08-22", "l_receiptdate": "1997-10-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nal foxes cajole above the blithely reg" }
+{ "l_orderkey": 1794, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33492.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rs above the accoun" }
+{ "l_orderkey": 2021, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6895.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-20", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts boost blithely. blithely reg" }
+{ "l_orderkey": 2081, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-06", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ual requests wake blithely above the" }
+{ "l_orderkey": 2211, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22656.84, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-05", "l_commitdate": "1994-09-13", "l_receiptdate": "1994-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ependencies " }
 { "l_orderkey": 2273, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 34477.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1997-01-19", "l_receiptdate": "1997-01-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "arefully f" }
+{ "l_orderkey": 3076, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43343.52, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-14", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-09-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " instructions h" }
+{ "l_orderkey": 3175, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13791.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-21", "l_commitdate": "1994-09-05", "l_receiptdate": "1994-11-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nt dependencies are quietly even " }
+{ "l_orderkey": 3783, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49254.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he furiously regular deposits. " }
+{ "l_orderkey": 4032, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9850.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-31", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully bol" }
+{ "l_orderkey": 4067, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 16746.36, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1992-12-29", "l_receiptdate": "1993-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r accounts. slyly special pa" }
 { "l_orderkey": 4224, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 47283.84, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-03", "l_commitdate": "1997-08-31", "l_receiptdate": "1997-10-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " final, regular asymptotes use alway" }
 { "l_orderkey": 5158, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17731.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "hely regular pa" }
+{ "l_orderkey": 5574, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18716.52, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-28", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old deposits int" }
 { "l_orderkey": 5891, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21671.76, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1993-02-18", "l_receiptdate": "1993-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iresias cajole deposits. special, ir" }
-{ "l_orderkey": 390, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 23641.92, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-18", "l_commitdate": "1998-05-19", "l_receiptdate": "1998-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. enticingly final depos" }
-{ "l_orderkey": 1794, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33492.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-29", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "rs above the accoun" }
-{ "l_orderkey": 4032, "l_partkey": 85, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9850.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-31", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " carefully bol" }
+{ "l_orderkey": 229, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19681.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "le. instructions use across the quickly fin" }
+{ "l_orderkey": 231, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45267.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix blithely. bold requests among the f" }
+{ "l_orderkey": 518, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-20", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " special requests. fluffily ironic re" }
 { "l_orderkey": 772, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9840.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-05-19", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " deposits cajole carefully instructions. t" }
 { "l_orderkey": 836, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17713.44, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y pending packages use alon" }
-{ "l_orderkey": 1510, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly brave theod" }
-{ "l_orderkey": 2086, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26570.16, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-04", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "theodolites haggle blithely blithe p" }
-{ "l_orderkey": 2471, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36410.96, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts mold blithely carefully express depo" }
-{ "l_orderkey": 3234, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-15", "l_receiptdate": "1996-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d-- fluffily special packag" }
-{ "l_orderkey": 3332, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27554.24, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the carefully special multipl" }
-{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32474.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully across the fur" }
-{ "l_orderkey": 4357, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-12-03", "l_receiptdate": "1997-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s. final, e" }
+{ "l_orderkey": 1190, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31490.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y final packages? slyly even" }
 { "l_orderkey": 1285, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32474.64, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-08", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-09-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ites affix" }
 { "l_orderkey": 1377, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25586.08, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-06-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular deposits. quickly regular acco" }
+{ "l_orderkey": 1510, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly brave theod" }
+{ "l_orderkey": 1920, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e blithely unusual foxes. brave packages" }
+{ "l_orderkey": 2017, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10824.88, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "gside of the slyly dogged dolp" }
 { "l_orderkey": 2049, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16729.36, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-04", "l_commitdate": "1996-03-01", "l_receiptdate": "1996-02-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "al, regular foxes. pending, " }
+{ "l_orderkey": 2086, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26570.16, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-04", "l_commitdate": "1995-01-14", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "theodolites haggle blithely blithe p" }
+{ "l_orderkey": 2471, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 36410.96, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts mold blithely carefully express depo" }
 { "l_orderkey": 2501, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3936.32, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-07-27", "l_receiptdate": "1997-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests. furiously final" }
+{ "l_orderkey": 2597, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-15", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending packages. enticingly fi" }
 { "l_orderkey": 3170, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 25586.08, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s engage furiously. " }
+{ "l_orderkey": 3234, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-15", "l_receiptdate": "1996-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d-- fluffily special packag" }
+{ "l_orderkey": 3296, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11808.96, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "y about the slyly bold pinto bea" }
+{ "l_orderkey": 3332, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27554.24, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the carefully special multipl" }
+{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28538.32, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly unusual i" }
+{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32474.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully across the fur" }
+{ "l_orderkey": 3936, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 34442.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly ironic requ" }
 { "l_orderkey": 4324, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21649.76, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-13", "l_commitdate": "1995-10-04", "l_receiptdate": "1995-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ke express, special ideas." }
+{ "l_orderkey": 4357, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-12-03", "l_receiptdate": "1997-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s. final, e" }
 { "l_orderkey": 4388, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27554.24, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-06-20", "l_receiptdate": "1996-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ove the ide" }
 { "l_orderkey": 4774, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44283.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " haggle busily afte" }
 { "l_orderkey": 5895, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 48219.92, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "permanent foxes. packages" }
-{ "l_orderkey": 231, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 45267.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "affix blithely. bold requests among the f" }
-{ "l_orderkey": 1190, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31490.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-08", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y final packages? slyly even" }
-{ "l_orderkey": 1920, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 49204.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-04", "l_receiptdate": "1998-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e blithely unusual foxes. brave packages" }
-{ "l_orderkey": 2017, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10824.88, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "gside of the slyly dogged dolp" }
-{ "l_orderkey": 2597, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23617.92, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-15", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending packages. enticingly fi" }
-{ "l_orderkey": 3296, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11808.96, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-08", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "y about the slyly bold pinto bea" }
-{ "l_orderkey": 3586, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28538.32, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " slyly unusual i" }
-{ "l_orderkey": 3936, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 34442.8, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "lly ironic requ" }
-{ "l_orderkey": 229, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19681.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "le. instructions use across the quickly fin" }
-{ "l_orderkey": 518, "l_partkey": 84, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22633.84, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-20", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-03-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " special requests. fluffily ironic re" }
-{ "l_orderkey": 192, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24577.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": ". carefully regular" }
-{ "l_orderkey": 358, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 44238.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-08", "l_commitdate": "1993-10-29", "l_receiptdate": "1993-12-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans. regular, unusual deposits sl" }
-{ "l_orderkey": 512, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xes. pinto beans cajole carefully; " }
-{ "l_orderkey": 1575, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "k excuses. pinto beans wake a" }
-{ "l_orderkey": 2916, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20644.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-11", "l_commitdate": "1996-02-21", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express ideas over the slyly even " }
-{ "l_orderkey": 3072, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38340.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es; slyly spe" }
-{ "l_orderkey": 5187, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "aggle never bold " }
-{ "l_orderkey": 5345, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2949.24, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-10-03", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ites wake carefully unusual " }
-{ "l_orderkey": 2403, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33424.72, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-19", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly bold re" }
-{ "l_orderkey": 5220, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26543.16, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-29", "l_receiptdate": "1992-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole blithely furiously iron" }
-{ "l_orderkey": 5635, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "cross the d" }
-{ "l_orderkey": 5765, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 40306.28, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " furiously. slyly sile" }
 { "l_orderkey": 32, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27526.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-23", "l_commitdate": "1995-08-27", "l_receiptdate": "1995-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sleep quickly. req" }
 { "l_orderkey": 68, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 26543.16, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ccounts. deposits use. furiously" }
+{ "l_orderkey": 192, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24577.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": ". carefully regular" }
 { "l_orderkey": 226, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 47187.84, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-11", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-06-19", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "efully silent packages. final deposit" }
+{ "l_orderkey": 358, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 44238.6, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-08", "l_commitdate": "1993-10-29", "l_receiptdate": "1993-12-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans. regular, unusual deposits sl" }
+{ "l_orderkey": 487, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1966.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oss the unusual pinto beans. reg" }
+{ "l_orderkey": 512, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xes. pinto beans cajole carefully; " }
 { "l_orderkey": 1120, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-10", "l_commitdate": "1998-02-01", "l_receiptdate": "1997-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ages haggle furiously " }
 { "l_orderkey": 1157, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3932.32, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-30", "l_receiptdate": "1998-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ounts. ironic deposits" }
 { "l_orderkey": 1411, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45221.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-03", "l_commitdate": "1995-01-20", "l_receiptdate": "1995-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly daring instructions" }
 { "l_orderkey": 1573, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15729.28, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-15", "l_commitdate": "1993-03-16", "l_receiptdate": "1993-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ely. furiously final requests wake slyl" }
-{ "l_orderkey": 3040, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13763.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle carefully. express hocke" }
-{ "l_orderkey": 3207, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7864.64, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-13", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. final pint" }
-{ "l_orderkey": 3814, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46204.76, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual requests. bli" }
-{ "l_orderkey": 3936, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41289.36, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "elets wake amo" }
-{ "l_orderkey": 5218, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-04", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-08-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "k theodolites. express, even id" }
-{ "l_orderkey": 5347, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47187.84, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-03-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "equests are slyly. blithely regu" }
-{ "l_orderkey": 487, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1966.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oss the unusual pinto beans. reg" }
+{ "l_orderkey": 1575, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9830.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-11-20", "l_receiptdate": "1996-01-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "k excuses. pinto beans wake a" }
 { "l_orderkey": 1958, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37357.04, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-09", "l_commitdate": "1995-11-26", "l_receiptdate": "1995-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "yly. slyly regular courts use silentl" }
+{ "l_orderkey": 2403, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33424.72, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-06-19", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " slyly bold re" }
+{ "l_orderkey": 2916, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20644.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-11", "l_commitdate": "1996-02-21", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express ideas over the slyly even " }
+{ "l_orderkey": 3040, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13763.12, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-13", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " haggle carefully. express hocke" }
+{ "l_orderkey": 3072, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38340.12, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-06-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es; slyly spe" }
+{ "l_orderkey": 3207, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7864.64, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-13", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. final pint" }
 { "l_orderkey": 3269, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 36373.96, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-14", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "egular requests. carefully un" }
 { "l_orderkey": 3750, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-06-25", "l_receiptdate": "1995-08-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "l dolphins against the slyly" }
+{ "l_orderkey": 3814, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 46204.76, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual requests. bli" }
+{ "l_orderkey": 3936, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 41289.36, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-01-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "elets wake amo" }
 { "l_orderkey": 4067, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 16712.36, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts affix. regular, regular requests s" }
 { "l_orderkey": 4738, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 27526.24, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-09", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-06-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e furiously ironic excuses. care" }
 { "l_orderkey": 4902, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-12", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-11-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "daring foxes? even, bold requests wake f" }
+{ "l_orderkey": 5187, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 983.08, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-08-24", "l_receiptdate": "1997-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "aggle never bold " }
+{ "l_orderkey": 5218, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-04", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-08-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "k theodolites. express, even id" }
+{ "l_orderkey": 5220, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26543.16, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-29", "l_receiptdate": "1992-10-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s cajole blithely furiously iron" }
+{ "l_orderkey": 5345, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2949.24, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-10-03", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ites wake carefully unusual " }
+{ "l_orderkey": 5347, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 47187.84, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-03-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "equests are slyly. blithely regu" }
 { "l_orderkey": 5443, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39323.2, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-12-10", "l_receiptdate": "1997-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "n courts. special re" }
-{ "l_orderkey": 931, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37319.04, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-02-24", "l_receiptdate": "1993-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly final packages integrate carefully" }
-{ "l_orderkey": 1444, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly among the bol" }
-{ "l_orderkey": 2241, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-16", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-08-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ", express deposits. pear" }
-{ "l_orderkey": 2595, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 30444.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-02-10", "l_receiptdate": "1996-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tipliers w" }
-{ "l_orderkey": 2976, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21605.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ncies kindle furiously. carefull" }
-{ "l_orderkey": 3360, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38301.12, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. blithely express pinto bean" }
-{ "l_orderkey": 4387, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2946.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pinto beans " }
-{ "l_orderkey": 5606, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29462.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " nag always. blithely express packages " }
+{ "l_orderkey": 5635, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42272.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-12", "l_commitdate": "1992-09-29", "l_receiptdate": "1992-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "cross the d" }
+{ "l_orderkey": 5765, "l_partkey": 83, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 40306.28, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " furiously. slyly sile" }
 { "l_orderkey": 481, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10802.88, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-12", "l_commitdate": "1992-11-17", "l_receiptdate": "1993-02-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "eful attai" }
 { "l_orderkey": 771, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 22587.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "cajole besides the quickly ironic pin" }
-{ "l_orderkey": 2723, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-24", "l_commitdate": "1995-11-15", "l_receiptdate": "1996-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bold foxes are bold packages. regular, fin" }
-{ "l_orderkey": 4261, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3928.32, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-11", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ackages unwind furiously fluff" }
-{ "l_orderkey": 4324, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13749.12, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages nag express excuses. qui" }
-{ "l_orderkey": 4933, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ctions nag final instructions. accou" }
-{ "l_orderkey": 5831, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5892.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts nag pendin" }
+{ "l_orderkey": 931, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 37319.04, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-02-24", "l_receiptdate": "1993-03-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "usly final packages integrate carefully" }
+{ "l_orderkey": 1444, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-01-15", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly among the bol" }
 { "l_orderkey": 1504, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 41247.36, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep. carefully ironic excuses haggle quickl" }
 { "l_orderkey": 1830, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 35354.88, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-21", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slowly unusual orbits. carefull" }
+{ "l_orderkey": 2241, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-16", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-08-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ", express deposits. pear" }
+{ "l_orderkey": 2468, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4910.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-28", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-07-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " sleep fluffily acc" }
+{ "l_orderkey": 2595, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 30444.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-02-10", "l_receiptdate": "1996-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tipliers w" }
+{ "l_orderkey": 2723, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11784.96, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-24", "l_commitdate": "1995-11-15", "l_receiptdate": "1996-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bold foxes are bold packages. regular, fin" }
+{ "l_orderkey": 2976, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21605.76, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ncies kindle furiously. carefull" }
+{ "l_orderkey": 3360, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38301.12, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-20", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. blithely express pinto bean" }
+{ "l_orderkey": 4261, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3928.32, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-11", "l_commitdate": "1992-12-18", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ackages unwind furiously fluff" }
+{ "l_orderkey": 4324, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13749.12, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " packages nag express excuses. qui" }
+{ "l_orderkey": 4387, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2946.24, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-17", "l_commitdate": "1995-12-28", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pinto beans " }
+{ "l_orderkey": 4611, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49104.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "l platelets. " }
 { "l_orderkey": 4867, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6874.56, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-08-17", "l_receiptdate": "1992-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e carefully even packages. slyly ironic i" }
+{ "l_orderkey": 4933, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1964.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-29", "l_receiptdate": "1995-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ctions nag final instructions. accou" }
 { "l_orderkey": 5122, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42229.44, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-31", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the carefully special foxes. idle," }
 { "l_orderkey": 5186, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34372.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-11-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sly silent pack" }
-{ "l_orderkey": 2468, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4910.4, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-28", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-07-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " sleep fluffily acc" }
-{ "l_orderkey": 4611, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 49104.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "l platelets. " }
 { "l_orderkey": 5317, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28480.32, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-28", "l_commitdate": "1994-11-27", "l_receiptdate": "1994-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "oss the carefull" }
-{ "l_orderkey": 2240, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-11", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-04-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss thinly deposits. blithely bold package" }
-{ "l_orderkey": 3394, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13735.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e furiously final theodolites. furio" }
-{ "l_orderkey": 4514, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8829.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "wly. quick" }
-{ "l_orderkey": 5248, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38262.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-04", "l_receiptdate": "1995-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yly even accounts. spe" }
+{ "l_orderkey": 5606, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29462.4, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1997-01-26", "l_receiptdate": "1997-02-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " nag always. blithely express packages " }
+{ "l_orderkey": 5831, "l_partkey": 82, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5892.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-29", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly ironic accounts nag pendin" }
 { "l_orderkey": 611, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 981.08, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-02-26", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ts. pending platelets aff" }
+{ "l_orderkey": 1221, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y slyly above the slyly unusual ideas" }
 { "l_orderkey": 1312, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8829.72, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-19", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-07-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". furiously " }
 { "l_orderkey": 1347, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44148.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-09-03", "l_receiptdate": "1997-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ages wake around t" }
-{ "l_orderkey": 3238, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 981.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-05-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "wake alongs" }
-{ "l_orderkey": 3364, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2943.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c theodolites. blithely ir" }
-{ "l_orderkey": 3430, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-01-28", "l_receiptdate": "1995-02-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "egular instruction" }
-{ "l_orderkey": 3875, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23545.92, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ecial packages. " }
-{ "l_orderkey": 1221, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y slyly above the slyly unusual ideas" }
+{ "l_orderkey": 2177, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22564.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he silent foxes. iro" }
+{ "l_orderkey": 2240, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-11", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-04-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss thinly deposits. blithely bold package" }
 { "l_orderkey": 2567, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 45129.68, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-02", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "efully pending epitaphs. carefully reg" }
 { "l_orderkey": 2915, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-24", "l_receiptdate": "1994-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "into beans dazzle alongside of" }
-{ "l_orderkey": 5027, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49054.0, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " beans dazzle according to the fluffi" }
-{ "l_orderkey": 5217, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46110.76, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-24", "l_commitdate": "1995-12-25", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic packages i" }
-{ "l_orderkey": 2177, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22564.84, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he silent foxes. iro" }
+{ "l_orderkey": 3238, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 981.08, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-04-18", "l_receiptdate": "1993-05-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "wake alongs" }
+{ "l_orderkey": 3364, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2943.24, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c theodolites. blithely ir" }
+{ "l_orderkey": 3394, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13735.12, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-07-02", "l_receiptdate": "1996-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e furiously final theodolites. furio" }
+{ "l_orderkey": 3430, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 31394.56, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1995-01-28", "l_receiptdate": "1995-02-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "egular instruction" }
 { "l_orderkey": 3687, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1962.16, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-03-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " express requests. slyly regular depend" }
+{ "l_orderkey": 3875, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23545.92, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ecial packages. " }
+{ "l_orderkey": 4514, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8829.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-07-01", "l_receiptdate": "1994-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "wly. quick" }
 { "l_orderkey": 4675, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17659.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-03-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cajole unusual dep" }
 { "l_orderkey": 4995, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 42186.44, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-03-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts. blithely silent ideas after t" }
+{ "l_orderkey": 5027, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 49054.0, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " beans dazzle according to the fluffi" }
+{ "l_orderkey": 5217, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 46110.76, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-24", "l_commitdate": "1995-12-25", "l_receiptdate": "1995-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic packages i" }
+{ "l_orderkey": 5248, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 38262.12, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-04", "l_receiptdate": "1995-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yly even accounts. spe" }
 { "l_orderkey": 5954, "l_partkey": 81, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39243.2, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-30", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "iously ironic deposits after" }
+{ "l_orderkey": 7, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34302.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-16", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "jole. excuses wake carefully alongside of " }
 { "l_orderkey": 483, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22541.84, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-20", "l_commitdate": "1995-08-11", "l_receiptdate": "1995-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "requests was quickly against th" }
-{ "l_orderkey": 807, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9800.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-01-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously final depths sleep a" }
-{ "l_orderkey": 1827, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23521.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al gifts! re" }
-{ "l_orderkey": 3652, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 980.08, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " bold dependencies sublate. r" }
+{ "l_orderkey": 644, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6860.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular requests are blithely. slyly" }
 { "l_orderkey": 736, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22541.84, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-08", "l_commitdate": "1998-08-27", "l_receiptdate": "1998-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "k accounts are carefully" }
+{ "l_orderkey": 807, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9800.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-01-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "furiously final depths sleep a" }
+{ "l_orderkey": 864, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33322.72, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously ironic platelets! " }
 { "l_orderkey": 962, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2940.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ag furiously. even pa" }
+{ "l_orderkey": 1121, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 37.0, "l_extendedprice": 36262.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-04", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "special packages. fluffily final requests s" }
+{ "l_orderkey": 1827, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23521.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-07", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al gifts! re" }
 { "l_orderkey": 1888, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 48023.92, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-01-14", "l_receiptdate": "1994-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lar accounts haggle carefu" }
-{ "l_orderkey": 3136, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 28422.32, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-16", "l_commitdate": "1994-10-03", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "latelets. final " }
-{ "l_orderkey": 3877, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43123.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-07-15", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "elets. quickly regular accounts caj" }
-{ "l_orderkey": 4422, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-07-16", "l_receiptdate": "1995-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ructions wake slyly al" }
-{ "l_orderkey": 5030, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss excuses serve bli" }
 { "l_orderkey": 2245, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32342.64, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-07-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across the express reques" }
+{ "l_orderkey": 2500, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40183.28, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s could have to integrate after the " }
 { "l_orderkey": 2756, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46063.76, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-05-25", "l_receiptdate": "1994-05-13", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "e final, f" }
+{ "l_orderkey": 3136, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 28422.32, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-16", "l_commitdate": "1994-10-03", "l_receiptdate": "1994-12-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "latelets. final " }
 { "l_orderkey": 3623, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31362.56, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-03-15", "l_receiptdate": "1997-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " courts. furiously regular ideas b" }
+{ "l_orderkey": 3652, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 980.08, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " bold dependencies sublate. r" }
+{ "l_orderkey": 3750, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-06-06", "l_receiptdate": "1995-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss, ironic requests! fur" }
+{ "l_orderkey": 3877, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 43123.52, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-07", "l_commitdate": "1993-07-15", "l_receiptdate": "1993-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "elets. quickly regular accounts caj" }
 { "l_orderkey": 4295, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29402.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "yly ironic frets. pending foxes after " }
+{ "l_orderkey": 4422, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-07-16", "l_receiptdate": "1995-09-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ructions wake slyly al" }
 { "l_orderkey": 4868, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33322.72, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-06-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "en instructions about th" }
+{ "l_orderkey": 4994, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46063.76, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sts. blithely close ideas sleep quic" }
+{ "l_orderkey": 5030, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-22", "l_commitdate": "1998-07-25", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss excuses serve bli" }
 { "l_orderkey": 5090, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29402.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-04", "l_commitdate": "1997-04-14", "l_receiptdate": "1997-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "osits nag slyly. fluffily ex" }
 { "l_orderkey": 5184, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 48023.92, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "thlessly closely even reque" }
 { "l_orderkey": 5217, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 49004.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-26", "l_commitdate": "1995-11-21", "l_receiptdate": "1996-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s. express, express accounts c" }
-{ "l_orderkey": 7, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 34302.8, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-16", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "jole. excuses wake carefully alongside of " }
-{ "l_orderkey": 644, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6860.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular requests are blithely. slyly" }
-{ "l_orderkey": 864, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33322.72, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-11-04", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously ironic platelets! " }
-{ "l_orderkey": 1121, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 37.0, "l_extendedprice": 36262.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-27", "l_commitdate": "1997-03-04", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "special packages. fluffily final requests s" }
-{ "l_orderkey": 2500, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 40183.28, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s could have to integrate after the " }
-{ "l_orderkey": 3750, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-17", "l_commitdate": "1995-06-06", "l_receiptdate": "1995-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ss, ironic requests! fur" }
-{ "l_orderkey": 4994, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 46063.76, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-10-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sts. blithely close ideas sleep quic" }
 { "l_orderkey": 5346, "l_partkey": 80, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 41.0, "l_extendedprice": 40183.28, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-10", "l_commitdate": "1994-02-15", "l_receiptdate": "1994-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "fully close instructi" }
 { "l_orderkey": 5477, "l_partkey": 80, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19601.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-21", "l_commitdate": "1998-02-09", "l_receiptdate": "1998-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "platelets about the ironic" }
 { "l_orderkey": 5543, "l_partkey": 80, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 31362.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-14", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ully around the " }
+{ "l_orderkey": 229, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27413.96, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final, regular requests. platel" }
 { "l_orderkey": 288, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18602.33, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-03-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "deposits. blithely quick courts ar" }
 { "l_orderkey": 450, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1958.14, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-05-21", "l_receiptdate": "1995-03-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "y even pinto beans; qui" }
 { "l_orderkey": 482, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18602.33, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-25", "l_receiptdate": "1996-04-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts hinder carefully silent requests" }
-{ "l_orderkey": 3109, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " sleep slyly according to t" }
-{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-25", "l_receiptdate": "1996-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uctions wake fluffily. care" }
-{ "l_orderkey": 4389, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38183.73, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual, final excuses cajole carefully " }
-{ "l_orderkey": 4997, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43079.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-07-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r escapades ca" }
-{ "l_orderkey": 5377, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-21", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely ironic theodolites are care" }
-{ "l_orderkey": 229, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27413.96, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final, regular requests. platel" }
 { "l_orderkey": 514, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20560.47, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-09", "l_commitdate": "1996-05-15", "l_receiptdate": "1996-07-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s sleep quickly blithely" }
+{ "l_orderkey": 613, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5874.42, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-09", "l_receiptdate": "1995-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic deposits eat " }
 { "l_orderkey": 1252, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-05", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "onic pinto beans haggle furiously " }
 { "l_orderkey": 1316, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14686.05, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-12", "l_commitdate": "1994-03-02", "l_receiptdate": "1994-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "se. furiously final depo" }
-{ "l_orderkey": 2211, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-21", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly final" }
-{ "l_orderkey": 2657, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24476.75, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly pinto beans. final " }
-{ "l_orderkey": 4069, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21539.54, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-08-04", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts. slyly special instruction" }
-{ "l_orderkey": 4418, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-05-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "luffily across the unusual ideas. reque" }
-{ "l_orderkey": 4929, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unts against " }
-{ "l_orderkey": 5344, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-07-26", "l_receiptdate": "1998-11-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "thely express packages" }
-{ "l_orderkey": 5894, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-09-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " asymptotes among the blithely silent " }
-{ "l_orderkey": 1828, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13706.98, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final packages along the carefully bold" }
-{ "l_orderkey": 2690, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 34267.45, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y silent pinto be" }
-{ "l_orderkey": 3141, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8811.63, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly ironic, pendi" }
-{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "as sleep carefully into the caref" }
-{ "l_orderkey": 3555, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14686.05, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y across the pending a" }
-{ "l_orderkey": 4608, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 48953.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " wake closely. even decoys haggle above" }
-{ "l_orderkey": 5986, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e fluffily ironic ideas. silent " }
-{ "l_orderkey": 613, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5874.42, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-08-09", "l_receiptdate": "1995-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y ironic deposits eat " }
 { "l_orderkey": 1669, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23497.68, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " regular, final deposits use quick" }
+{ "l_orderkey": 1828, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13706.98, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final packages along the carefully bold" }
 { "l_orderkey": 1957, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48953.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-09-28", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "gainst the re" }
 { "l_orderkey": 1988, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-25", "l_commitdate": "1995-12-15", "l_receiptdate": "1996-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic dolphins haggl" }
+{ "l_orderkey": 2211, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-21", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y slyly final" }
 { "l_orderkey": 2466, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29372.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". fluffily even pinto beans are idly. f" }
+{ "l_orderkey": 2657, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24476.75, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly pinto beans. final " }
+{ "l_orderkey": 2690, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 34267.45, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "y silent pinto be" }
+{ "l_orderkey": 3109, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-16", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " sleep slyly according to t" }
+{ "l_orderkey": 3141, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8811.63, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-11", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "uickly ironic, pendi" }
 { "l_orderkey": 3234, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 44058.15, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-15", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " express packages are carefully. f" }
+{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-04", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-06-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "as sleep carefully into the caref" }
+{ "l_orderkey": 3425, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-25", "l_receiptdate": "1996-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "uctions wake fluffily. care" }
+{ "l_orderkey": 3555, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14686.05, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-13", "l_commitdate": "1996-09-01", "l_receiptdate": "1996-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y across the pending a" }
 { "l_orderkey": 3969, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 45037.22, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "fully final requests sleep stealthily. care" }
 { "l_orderkey": 4038, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23497.68, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-01", "l_commitdate": "1996-04-05", "l_receiptdate": "1996-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ake quickly after the final, ironic ac" }
+{ "l_orderkey": 4069, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21539.54, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-05", "l_commitdate": "1992-08-04", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts. slyly special instruction" }
 { "l_orderkey": 4293, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 44058.15, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-11-06", "l_receiptdate": "1996-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar ideas use carefully" }
+{ "l_orderkey": 4389, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 38183.73, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-08", "l_commitdate": "1994-06-04", "l_receiptdate": "1994-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual, final excuses cajole carefully " }
+{ "l_orderkey": 4418, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2937.21, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-08", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-05-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "luffily across the unusual ideas. reque" }
+{ "l_orderkey": 4608, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 48953.5, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " wake closely. even decoys haggle above" }
+{ "l_orderkey": 4929, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "unts against " }
+{ "l_orderkey": 4997, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 43079.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-06-12", "l_receiptdate": "1998-07-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "r escapades ca" }
 { "l_orderkey": 5094, "l_partkey": 79, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20560.47, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-26", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-08-16", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithely furiously final re" }
 { "l_orderkey": 5155, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 38183.73, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-09-01", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l dolphins nag caref" }
-{ "l_orderkey": 676, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19561.4, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-01", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "riously around the blithely " }
-{ "l_orderkey": 771, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-08-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "packages affix slyly about the quickly " }
-{ "l_orderkey": 1665, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly final p" }
-{ "l_orderkey": 1764, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly final foxes wake blithely even requests" }
-{ "l_orderkey": 2145, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "alongside of the slyly final" }
-{ "l_orderkey": 4359, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-05-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " fluffily ironic, bold pac" }
-{ "l_orderkey": 4514, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake furiously. carefully regular requests" }
-{ "l_orderkey": 5025, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly silent deposits boost busily again" }
-{ "l_orderkey": 5091, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48903.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al dependencies. r" }
-{ "l_orderkey": 2178, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2934.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-01-23", "l_receiptdate": "1997-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " permanentl" }
-{ "l_orderkey": 2657, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41078.94, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-23", "l_commitdate": "1995-11-22", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ckly slyly even accounts. platelets x-ray" }
-{ "l_orderkey": 5604, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-06-23", "l_receiptdate": "1998-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly final realms wake blit" }
+{ "l_orderkey": 5344, "l_partkey": 79, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 36225.59, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-07-26", "l_receiptdate": "1998-11-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "thely express packages" }
+{ "l_orderkey": 5377, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39162.8, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-21", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lithely ironic theodolites are care" }
+{ "l_orderkey": 5894, "l_partkey": 79, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46995.36, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-04", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-09-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " asymptotes among the blithely silent " }
+{ "l_orderkey": 5986, "l_partkey": 79, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25455.82, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e fluffily ironic ideas. silent " }
 { "l_orderkey": 97, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18583.33, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-14", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "gifts. furiously ironic packages cajole. " }
 { "l_orderkey": 129, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 21517.54, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e. fluffily regular " }
+{ "l_orderkey": 353, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44991.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ironic dolphins " }
+{ "l_orderkey": 676, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19561.4, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-01", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "riously around the blithely " }
+{ "l_orderkey": 771, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-08-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "packages affix slyly about the quickly " }
 { "l_orderkey": 1187, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 39122.8, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-03-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ar, brave deposits nag blithe" }
+{ "l_orderkey": 1251, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-07", "l_receiptdate": "1997-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic Tiresias are slyly furio" }
 { "l_orderkey": 1281, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 42057.01, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-28", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "final accounts. final packages slee" }
 { "l_orderkey": 1380, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14671.05, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-08-12", "l_receiptdate": "1996-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "riously ironic foxes aff" }
-{ "l_orderkey": 2210, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " requests wake enticingly final" }
-{ "l_orderkey": 3077, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "luffily close depende" }
-{ "l_orderkey": 3078, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20539.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e fluffily. " }
-{ "l_orderkey": 353, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44991.22, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ironic dolphins " }
-{ "l_orderkey": 1251, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-07", "l_receiptdate": "1997-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic Tiresias are slyly furio" }
+{ "l_orderkey": 1665, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly final p" }
+{ "l_orderkey": 1764, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-06", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ly final foxes wake blithely even requests" }
 { "l_orderkey": 2022, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-04", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " orbits haggle fluffily fl" }
+{ "l_orderkey": 2145, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-13", "l_receiptdate": "1992-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "alongside of the slyly final" }
 { "l_orderkey": 2150, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 25429.82, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-21", "l_commitdate": "1994-08-05", "l_receiptdate": "1994-06-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". always unusual packages" }
+{ "l_orderkey": 2178, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2934.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-07", "l_commitdate": "1997-01-23", "l_receiptdate": "1997-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " permanentl" }
+{ "l_orderkey": 2210, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35210.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " requests wake enticingly final" }
 { "l_orderkey": 2240, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 23473.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ng the silent accounts. slyly ironic t" }
 { "l_orderkey": 2532, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48903.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-13", "l_commitdate": "1996-01-01", "l_receiptdate": "1995-11-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "yly after the fluffily regul" }
+{ "l_orderkey": 2657, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 41078.94, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-23", "l_commitdate": "1995-11-22", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ckly slyly even accounts. platelets x-ray" }
 { "l_orderkey": 2882, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-04", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "rding to the regu" }
 { "l_orderkey": 2918, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23473.68, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-20", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " quickly. express requests haggle careful" }
+{ "l_orderkey": 3077, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12714.91, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-09-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "luffily close depende" }
+{ "l_orderkey": 3078, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20539.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e fluffily. " }
 { "l_orderkey": 3719, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 18583.33, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he regular ideas integrate acros" }
+{ "l_orderkey": 4359, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 978.07, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-27", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-05-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " fluffily ironic, bold pac" }
+{ "l_orderkey": 4514, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-19", "l_commitdate": "1994-06-25", "l_receiptdate": "1994-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake furiously. carefully regular requests" }
 { "l_orderkey": 4704, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13692.98, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-27", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " above the slyly final requests. quickly " }
+{ "l_orderkey": 5025, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-04", "l_commitdate": "1997-04-29", "l_receiptdate": "1997-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly silent deposits boost busily again" }
 { "l_orderkey": 5088, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 22495.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-03", "l_commitdate": "1993-03-07", "l_receiptdate": "1993-03-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "cording to the fluffily expr" }
+{ "l_orderkey": 5091, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48903.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "al dependencies. r" }
 { "l_orderkey": 5126, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 22495.61, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-02", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular, blithe packages." }
 { "l_orderkey": 5538, "l_partkey": 78, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8802.63, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "encies across the blithely fina" }
+{ "l_orderkey": 5604, "l_partkey": 78, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9780.7, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-03", "l_commitdate": "1998-06-23", "l_receiptdate": "1998-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly final realms wake blit" }
 { "l_orderkey": 5829, "l_partkey": 78, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 27.0, "l_extendedprice": 26407.89, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-31", "l_receiptdate": "1997-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ns about the excuses are c" }
 { "l_orderkey": 5895, "l_partkey": 78, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14671.05, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "silent package" }
 { "l_orderkey": 451, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27357.96, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-16", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " theodolites. even cou" }
-{ "l_orderkey": 5824, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39082.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "he final packag" }
 { "l_orderkey": 484, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 46899.36, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-05", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-03-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l, bold packages? even mult" }
 { "l_orderkey": 1157, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44945.22, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-19", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-04-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "slyly regular excuses. accounts" }
+{ "l_orderkey": 1411, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29312.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ious foxes wake courts. caref" }
 { "l_orderkey": 1635, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39082.8, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uriously up the ironic deposits. slyly i" }
+{ "l_orderkey": 1891, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-20", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ests along" }
+{ "l_orderkey": 2497, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14656.05, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1992-11-20", "l_receiptdate": "1993-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sly against the" }
 { "l_orderkey": 3494, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29312.1, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-07-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ns are quickly regular, " }
+{ "l_orderkey": 3815, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2931.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular, express ideas. ironic, final dep" }
 { "l_orderkey": 4166, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4885.35, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-06-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "hely unusual packages are above the f" }
 { "l_orderkey": 4293, "l_partkey": 77, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48853.5, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " special deposits. furiousl" }
-{ "l_orderkey": 4900, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32243.31, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "nto beans nag slyly reg" }
-{ "l_orderkey": 1891, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-20", "l_commitdate": "1995-01-16", "l_receiptdate": "1995-01-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ests along" }
-{ "l_orderkey": 5059, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "enly. requests doze. express, close pa" }
-{ "l_orderkey": 5477, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20518.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-28", "l_commitdate": "1998-02-15", "l_receiptdate": "1998-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "blate slyly. silent" }
-{ "l_orderkey": 1411, "l_partkey": 77, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29312.1, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1995-02-01", "l_receiptdate": "1995-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ious foxes wake courts. caref" }
-{ "l_orderkey": 2497, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14656.05, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-23", "l_commitdate": "1992-11-20", "l_receiptdate": "1993-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sly against the" }
-{ "l_orderkey": 3815, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2931.21, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-15", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "egular, express ideas. ironic, final dep" }
 { "l_orderkey": 4546, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3908.28, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly pending dependencies along the furio" }
 { "l_orderkey": 4708, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31266.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-12", "l_commitdate": "1994-11-14", "l_receiptdate": "1994-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the accounts. e" }
+{ "l_orderkey": 4900, "l_partkey": 77, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32243.31, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-18", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "nto beans nag slyly reg" }
 { "l_orderkey": 4929, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 31266.24, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "usly at the blithely pending pl" }
+{ "l_orderkey": 5059, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43968.15, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-28", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "enly. requests doze. express, close pa" }
+{ "l_orderkey": 5477, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20518.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-28", "l_commitdate": "1998-02-15", "l_receiptdate": "1998-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "blate slyly. silent" }
 { "l_orderkey": 5702, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42991.08, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-11-25", "l_receiptdate": "1994-01-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lites. carefully final requests doze b" }
+{ "l_orderkey": 5824, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39082.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "he final packag" }
 { "l_orderkey": 5889, "l_partkey": 77, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16610.19, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-01", "l_commitdate": "1995-08-12", "l_receiptdate": "1995-07-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "blithely pending packages. flu" }
-{ "l_orderkey": 1408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10736.77, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y even accounts thrash care" }
-{ "l_orderkey": 1634, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1952.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. carefully regular asymptotes wake" }
-{ "l_orderkey": 3649, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39042.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffy somas sleep quickly-- ironic de" }
-{ "l_orderkey": 3751, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38066.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to beans. pending, express packages c" }
-{ "l_orderkey": 4836, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13664.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites. unusual, bold dolphins ar" }
-{ "l_orderkey": 5408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33186.38, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "requests detect blithely a" }
-{ "l_orderkey": 1924, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14641.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1996-11-13", "l_receiptdate": "1997-01-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he package" }
-{ "l_orderkey": 2272, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11712.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-04-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " accounts cajole. quickly b" }
-{ "l_orderkey": 4066, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7808.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-05-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts. special pinto beans" }
-{ "l_orderkey": 4262, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 29282.1, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "tes after the carefully" }
-{ "l_orderkey": 4963, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15617.12, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " carefully slyly u" }
 { "l_orderkey": 486, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35138.52, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "deposits around the quickly regular packa" }
-{ "l_orderkey": 805, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25377.82, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-09-24", "l_receiptdate": "1995-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". ironic deposits sleep across " }
-{ "l_orderkey": 1120, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20497.47, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s: fluffily even packages c" }
-{ "l_orderkey": 2214, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26353.89, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "x fluffily along the even packages-- " }
-{ "l_orderkey": 2245, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42947.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully even sheaves" }
-{ "l_orderkey": 4966, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9760.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. carefully pending requests" }
 { "l_orderkey": 676, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 32210.31, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "as wake slyly furiously close pinto b" }
 { "l_orderkey": 804, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42947.08, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-06", "l_commitdate": "1993-04-13", "l_receiptdate": "1993-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly final deposits? special " }
+{ "l_orderkey": 805, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25377.82, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-28", "l_commitdate": "1995-09-24", "l_receiptdate": "1995-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": ". ironic deposits sleep across " }
+{ "l_orderkey": 1120, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20497.47, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s: fluffily even packages c" }
+{ "l_orderkey": 1408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10736.77, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y even accounts thrash care" }
 { "l_orderkey": 1607, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 33186.38, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-06", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-01-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " quickly above the " }
+{ "l_orderkey": 1634, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1952.14, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-12-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly. carefully regular asymptotes wake" }
+{ "l_orderkey": 1924, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 14641.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1996-11-13", "l_receiptdate": "1997-01-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "he package" }
+{ "l_orderkey": 2214, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26353.89, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-06-07", "l_receiptdate": "1998-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "x fluffily along the even packages-- " }
+{ "l_orderkey": 2245, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42947.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "refully even sheaves" }
+{ "l_orderkey": 2272, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11712.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-04-23", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " accounts cajole. quickly b" }
 { "l_orderkey": 2565, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 26.0, "l_extendedprice": 25377.82, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ites wake. ironic acco" }
+{ "l_orderkey": 3649, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 39042.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffy somas sleep quickly-- ironic de" }
+{ "l_orderkey": 3751, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 38066.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-01", "l_commitdate": "1994-06-01", "l_receiptdate": "1994-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to beans. pending, express packages c" }
+{ "l_orderkey": 4066, "l_partkey": 76, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7808.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-24", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-05-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts. special pinto beans" }
+{ "l_orderkey": 4262, "l_partkey": 76, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 29282.1, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "tes after the carefully" }
+{ "l_orderkey": 4836, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13664.98, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-02-06", "l_receiptdate": "1997-03-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites. unusual, bold dolphins ar" }
+{ "l_orderkey": 4963, "l_partkey": 76, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15617.12, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " carefully slyly u" }
+{ "l_orderkey": 4966, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9760.7, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " requests. carefully pending requests" }
+{ "l_orderkey": 5408, "l_partkey": 76, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 33186.38, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-22", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "requests detect blithely a" }
+{ "l_orderkey": 420, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11700.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c instructions are " }
 { "l_orderkey": 581, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29252.1, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-19", "l_commitdate": "1997-05-21", "l_receiptdate": "1997-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " regular ideas grow furio" }
 { "l_orderkey": 900, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23401.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-21", "l_commitdate": "1994-12-25", "l_receiptdate": "1994-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "-ray furiously un" }
-{ "l_orderkey": 1569, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-16", "l_commitdate": "1998-06-21", "l_receiptdate": "1998-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages. ironic, even excuses a" }
-{ "l_orderkey": 4000, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 42903.08, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-27", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "equests use blithely blithely bold d" }
-{ "l_orderkey": 4230, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-12", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-06-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nt instruct" }
-{ "l_orderkey": 4610, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25351.82, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-07-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " to the fluffily ironic requests h" }
-{ "l_orderkey": 5472, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-05-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e requests detect furiously. ruthlessly un" }
-{ "l_orderkey": 2944, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17551.26, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-07", "l_commitdate": "1997-10-26", "l_receiptdate": "1998-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " furiously slyl" }
-{ "l_orderkey": 3234, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15601.12, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ithely ironic accounts wake along t" }
-{ "l_orderkey": 3525, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30227.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-27", "l_receiptdate": "1996-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he careful" }
-{ "l_orderkey": 4708, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely. carefully sp" }
-{ "l_orderkey": 5062, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3900.28, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-14", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ke furiously express theodolites. " }
+{ "l_orderkey": 992, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 39977.87, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-14", "l_commitdate": "1998-02-04", "l_receiptdate": "1997-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eodolites cajole across the accounts." }
 { "l_orderkey": 1124, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 32177.31, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-19", "l_commitdate": "1998-09-17", "l_receiptdate": "1998-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "eposits sleep slyly. stealthily f" }
 { "l_orderkey": 1216, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46803.36, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-17", "l_commitdate": "1993-02-01", "l_receiptdate": "1993-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "symptotes use against th" }
+{ "l_orderkey": 1569, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-16", "l_commitdate": "1998-06-21", "l_receiptdate": "1998-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages. ironic, even excuses a" }
 { "l_orderkey": 1668, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40952.94, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ole carefully excuses. final" }
-{ "l_orderkey": 3014, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "es are. final braids nag slyly. fluff" }
-{ "l_orderkey": 3396, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31202.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-07", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits are slyly. final, bold foxes s" }
-{ "l_orderkey": 3494, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22426.61, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits nag " }
-{ "l_orderkey": 4103, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usly across the slyly busy accounts! fin" }
-{ "l_orderkey": 4227, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10725.77, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-02", "l_receiptdate": "1995-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l requests-- bold requests cajole dogg" }
-{ "l_orderkey": 4417, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-09-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ies across the furious" }
-{ "l_orderkey": 420, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11700.84, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c instructions are " }
-{ "l_orderkey": 992, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 39977.87, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-14", "l_commitdate": "1998-02-04", "l_receiptdate": "1997-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "eodolites cajole across the accounts." }
 { "l_orderkey": 1697, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5850.42, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1996-11-27", "l_receiptdate": "1997-01-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "accounts breach slyly even de" }
 { "l_orderkey": 1859, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17551.26, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e carefully a" }
 { "l_orderkey": 2082, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 35102.52, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "haggle furiously silent pinto beans" }
 { "l_orderkey": 2534, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 41928.01, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-25", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ngly final depos" }
 { "l_orderkey": 2791, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8775.63, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "pendencies. blithely bold patterns acr" }
+{ "l_orderkey": 2944, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17551.26, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-07", "l_commitdate": "1997-10-26", "l_receiptdate": "1998-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " furiously slyl" }
+{ "l_orderkey": 3014, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-12-18", "l_receiptdate": "1993-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "es are. final braids nag slyly. fluff" }
+{ "l_orderkey": 3234, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15601.12, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-10", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-06-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ithely ironic accounts wake along t" }
+{ "l_orderkey": 3396, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 31202.24, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-07", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "osits are slyly. final, bold foxes s" }
+{ "l_orderkey": 3494, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22426.61, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-06-04", "l_receiptdate": "1993-07-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "osits nag " }
+{ "l_orderkey": 3525, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30227.17, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-02-27", "l_receiptdate": "1996-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "he careful" }
+{ "l_orderkey": 4000, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 42903.08, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-27", "l_commitdate": "1992-02-18", "l_receiptdate": "1992-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "equests use blithely blithely bold d" }
+{ "l_orderkey": 4103, "l_partkey": 75, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-19", "l_commitdate": "1992-08-14", "l_receiptdate": "1992-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usly across the slyly busy accounts! fin" }
+{ "l_orderkey": 4227, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10725.77, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-02", "l_receiptdate": "1995-04-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "l requests-- bold requests cajole dogg" }
+{ "l_orderkey": 4230, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-12", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-06-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nt instruct" }
+{ "l_orderkey": 4417, "l_partkey": 75, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 27301.96, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-09-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ies across the furious" }
+{ "l_orderkey": 4610, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 25351.82, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-07-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " to the fluffily ironic requests h" }
+{ "l_orderkey": 4708, "l_partkey": 75, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4875.35, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-15", "l_commitdate": "1994-12-02", "l_receiptdate": "1994-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely. carefully sp" }
+{ "l_orderkey": 5062, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3900.28, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-06", "l_commitdate": "1992-12-14", "l_receiptdate": "1993-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ke furiously express theodolites. " }
+{ "l_orderkey": 5472, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 39002.8, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-05-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e requests detect furiously. ruthlessly un" }
 { "l_orderkey": 5892, "l_partkey": 75, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 22426.61, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-05-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " foxes nag slyly about the qui" }
+{ "l_orderkey": 65, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21429.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ideas. special, r" }
+{ "l_orderkey": 801, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-22", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " even asymptotes" }
+{ "l_orderkey": 929, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13636.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-11-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "gainst the" }
+{ "l_orderkey": 1156, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47729.43, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithely unusual in" }
 { "l_orderkey": 1888, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 37014.66, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1994-01-16", "l_receiptdate": "1993-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dazzle carefull" }
+{ "l_orderkey": 2208, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 39936.87, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-06-19", "l_receiptdate": "1995-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nd the furious, express dependencies." }
+{ "l_orderkey": 2245, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-19", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e requests sleep furiou" }
+{ "l_orderkey": 2374, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ets cajole fu" }
+{ "l_orderkey": 2528, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12662.91, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1995-01-20", "l_receiptdate": "1994-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ggle furiously. slyly final asympt" }
+{ "l_orderkey": 3046, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42859.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " are quickly. blithe" }
 { "l_orderkey": 3460, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2922.21, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "er quickly " }
 { "l_orderkey": 3587, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 22403.61, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "l multipliers sleep theodolites-- slyly " }
+{ "l_orderkey": 4004, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ncies. slyly pending dolphins sleep furio" }
+{ "l_orderkey": 4097, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48703.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-31", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. blithely pending" }
 { "l_orderkey": 4097, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-29", "l_commitdate": "1996-08-19", "l_receiptdate": "1996-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " even depend" }
 { "l_orderkey": 4100, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3896.28, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-04-29", "l_receiptdate": "1996-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly regular, bold requ" }
 { "l_orderkey": 4162, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-21", "l_commitdate": "1992-05-02", "l_receiptdate": "1992-03-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "elets. slyly regular i" }
-{ "l_orderkey": 4775, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 974.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "furiously ironic theodolite" }
-{ "l_orderkey": 5093, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14611.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-02", "l_commitdate": "1993-11-18", "l_receiptdate": "1994-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly among the unusual foxe" }
-{ "l_orderkey": 5831, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32144.31, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions wake. slyly sil" }
-{ "l_orderkey": 801, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-03-22", "l_receiptdate": "1992-03-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " even asymptotes" }
-{ "l_orderkey": 1156, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47729.43, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ithely unusual in" }
-{ "l_orderkey": 2208, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 39936.87, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-06-19", "l_receiptdate": "1995-09-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "nd the furious, express dependencies." }
-{ "l_orderkey": 3046, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42859.08, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " are quickly. blithe" }
 { "l_orderkey": 4262, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 43833.15, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-09-09", "l_receiptdate": "1996-11-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages boost. pending, even instruction" }
 { "l_orderkey": 4288, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31170.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-19", "l_commitdate": "1993-01-26", "l_receiptdate": "1993-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e blithely even instructions. speci" }
-{ "l_orderkey": 2374, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-19", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ets cajole fu" }
-{ "l_orderkey": 2528, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12662.91, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-27", "l_commitdate": "1995-01-20", "l_receiptdate": "1994-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ggle furiously. slyly final asympt" }
-{ "l_orderkey": 4004, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-13", "l_receiptdate": "1993-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ncies. slyly pending dolphins sleep furio" }
 { "l_orderkey": 4578, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9740.7, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-01", "l_commitdate": "1992-11-19", "l_receiptdate": "1993-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "uests. blithely unus" }
-{ "l_orderkey": 65, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21429.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-17", "l_commitdate": "1995-06-04", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ideas. special, r" }
-{ "l_orderkey": 929, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13636.98, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-11-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "gainst the" }
-{ "l_orderkey": 2245, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27273.96, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-19", "l_commitdate": "1993-07-27", "l_receiptdate": "1993-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e requests sleep furiou" }
-{ "l_orderkey": 4097, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48703.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-31", "l_commitdate": "1996-08-14", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular deposits. blithely pending" }
+{ "l_orderkey": 4775, "l_partkey": 74, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 974.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-28", "l_receiptdate": "1995-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "furiously ironic theodolite" }
+{ "l_orderkey": 5093, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14611.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-02", "l_commitdate": "1993-11-18", "l_receiptdate": "1994-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly among the unusual foxe" }
 { "l_orderkey": 5412, "l_partkey": 74, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 30196.17, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-23", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-04-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "t the accounts detect slyly about the c" }
 { "l_orderkey": 5606, "l_partkey": 74, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 44807.22, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-01", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-02-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ow requests wake around the regular accoun" }
+{ "l_orderkey": 5831, "l_partkey": 74, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 32144.31, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1997-01-18", "l_receiptdate": "1996-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions wake. slyly sil" }
 { "l_orderkey": 420, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 36003.59, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-12-13", "l_receiptdate": "1995-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "rbits. bold requests along the quickl" }
-{ "l_orderkey": 1637, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly final pinto beans. furiously" }
-{ "l_orderkey": 1762, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-11-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely brave" }
-{ "l_orderkey": 4167, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-11", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "xpress platelets. blithely " }
-{ "l_orderkey": 4994, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5838.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "grate carefully around th" }
 { "l_orderkey": 676, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-05", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ress, regular dep" }
-{ "l_orderkey": 3783, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35030.52, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular accounts" }
-{ "l_orderkey": 3840, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43788.15, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-12", "l_commitdate": "1998-10-12", "l_receiptdate": "1998-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "onic, even packages are. pe" }
-{ "l_orderkey": 4741, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas boost furiously slyly regular id" }
+{ "l_orderkey": 998, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-05", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-01-13", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "es sleep. regular dependencies use bl" }
+{ "l_orderkey": 1378, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10703.77, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely express hoc" }
 { "l_orderkey": 1379, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12649.91, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-08", "l_commitdate": "1998-07-13", "l_receiptdate": "1998-06-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ully across the furiously iron" }
+{ "l_orderkey": 1637, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly final pinto beans. furiously" }
+{ "l_orderkey": 1732, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-02", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ix carefully at the furiously regular pac" }
+{ "l_orderkey": 1761, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 47680.43, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y even packages promise" }
+{ "l_orderkey": 1762, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-02", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-11-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " blithely brave" }
+{ "l_orderkey": 1924, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6811.49, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "osits. even accounts nag furious" }
 { "l_orderkey": 1927, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14596.05, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " carefully regular requests sleep car" }
+{ "l_orderkey": 1958, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8757.63, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly. slyly bold " }
 { "l_orderkey": 2050, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-07-18", "l_receiptdate": "1994-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "tside the blithely pending packages eat f" }
 { "l_orderkey": 2215, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 32111.31, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites cajole b" }
 { "l_orderkey": 2757, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16542.19, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "er the furiously silent " }
-{ "l_orderkey": 4743, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20434.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake blithely against the packages. reg" }
-{ "l_orderkey": 4868, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle unusual, fluffy packages. foxes cajol" }
-{ "l_orderkey": 5601, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 36976.66, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-08", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ter the evenly final deposit" }
-{ "l_orderkey": 5824, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-01-07", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly express Ti" }
-{ "l_orderkey": 998, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-05", "l_commitdate": "1995-01-06", "l_receiptdate": "1995-01-13", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "es sleep. regular dependencies use bl" }
-{ "l_orderkey": 1378, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10703.77, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-09", "l_receiptdate": "1996-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " blithely express hoc" }
-{ "l_orderkey": 1732, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-02", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ix carefully at the furiously regular pac" }
-{ "l_orderkey": 1761, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 47680.43, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-08", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y even packages promise" }
-{ "l_orderkey": 1924, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6811.49, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "osits. even accounts nag furious" }
-{ "l_orderkey": 1958, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8757.63, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1995-12-17", "l_receiptdate": "1995-12-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly. slyly bold " }
 { "l_orderkey": 2790, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 12649.91, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "n deposits according to the regul" }
 { "l_orderkey": 2951, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20434.47, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nt instructions toward the f" }
+{ "l_orderkey": 3783, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 35030.52, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "egular accounts" }
+{ "l_orderkey": 3840, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43788.15, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-12", "l_commitdate": "1998-10-12", "l_receiptdate": "1998-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "onic, even packages are. pe" }
+{ "l_orderkey": 4167, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 973.07, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-11", "l_commitdate": "1998-08-14", "l_receiptdate": "1998-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "xpress platelets. blithely " }
 { "l_orderkey": 4614, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-01", "l_commitdate": "1996-06-24", "l_receiptdate": "1996-07-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "regular, even" }
-{ "l_orderkey": 1060, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 25273.82, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "quickly abo" }
-{ "l_orderkey": 2594, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6804.49, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arls cajole " }
-{ "l_orderkey": 2945, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular instructions" }
-{ "l_orderkey": 3558, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16525.19, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely unusual packa" }
-{ "l_orderkey": 3969, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 38882.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar requests cajole furiously blithely regu" }
-{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4860.35, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly along the ironic, fi" }
-{ "l_orderkey": 1441, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33050.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e carefully. blithely ironic dep" }
-{ "l_orderkey": 2435, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final accounts ar" }
-{ "l_orderkey": 2501, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19441.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. furiou" }
-{ "l_orderkey": 4005, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27217.96, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-01-14", "l_receiptdate": "1996-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y pending dependenc" }
-{ "l_orderkey": 4196, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-07-21", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " accounts. fu" }
-{ "l_orderkey": 4672, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ests. idle, regular ex" }
-{ "l_orderkey": 5313, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-06-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding packages use" }
-{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ke slyly against the carefully final req" }
+{ "l_orderkey": 4741, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 23353.68, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "deas boost furiously slyly regular id" }
+{ "l_orderkey": 4743, "l_partkey": 73, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20434.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-02", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake blithely against the packages. reg" }
+{ "l_orderkey": 4868, "l_partkey": 73, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45734.29, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "gle unusual, fluffy packages. foxes cajol" }
+{ "l_orderkey": 4994, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5838.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-04", "l_receiptdate": "1996-09-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "grate carefully around th" }
+{ "l_orderkey": 5601, "l_partkey": 73, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 36976.66, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-08", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ter the evenly final deposit" }
+{ "l_orderkey": 5824, "l_partkey": 73, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15569.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-01-07", "l_receiptdate": "1997-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "sly express Ti" }
 { "l_orderkey": 486, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-05-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " blithely final pinto " }
 { "l_orderkey": 800, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-09-25", "l_receiptdate": "1998-08-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "according to the bold, final dependencies " }
+{ "l_orderkey": 1060, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 25273.82, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "quickly abo" }
 { "l_orderkey": 1185, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7776.56, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-05", "l_commitdate": "1992-10-05", "l_receiptdate": "1992-12-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely according to the furiously regular r" }
 { "l_orderkey": 1222, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-12", "l_commitdate": "1993-03-14", "l_receiptdate": "1993-03-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "s print permanently unusual packages. " }
+{ "l_orderkey": 1441, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 33050.38, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-11", "l_receiptdate": "1997-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "e carefully. blithely ironic dep" }
 { "l_orderkey": 1446, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30134.17, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": ". slyly reg" }
 { "l_orderkey": 1477, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 30134.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-16", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " requests. fluffily final " }
-{ "l_orderkey": 2560, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34994.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts alongside of the excuses are " }
-{ "l_orderkey": 3013, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18469.33, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-18", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fluffily pending packages nag furiously al" }
-{ "l_orderkey": 3077, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24301.75, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lent account" }
-{ "l_orderkey": 3393, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46659.36, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-09-15", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " blithely final reques" }
-{ "l_orderkey": 4898, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42771.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y regular grouches about" }
-{ "l_orderkey": 5443, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37910.73, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-01", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gage carefully across the furiously" }
-{ "l_orderkey": 5540, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23329.68, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits! ironic depths may engage-- b" }
 { "l_orderkey": 1988, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34994.52, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-21", "l_commitdate": "1995-11-24", "l_receiptdate": "1996-01-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gular theodolites. " }
+{ "l_orderkey": 2435, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-06-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " final accounts ar" }
+{ "l_orderkey": 2501, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19441.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-07-01", "l_receiptdate": "1997-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "equests. furiou" }
+{ "l_orderkey": 2560, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34994.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "accounts alongside of the excuses are " }
+{ "l_orderkey": 2594, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6804.49, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-03-05", "l_receiptdate": "1993-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "arls cajole " }
 { "l_orderkey": 2756, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-05", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ular packages. regular deposi" }
 { "l_orderkey": 2821, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3888.28, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-19", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ual multipliers. final deposits cajol" }
+{ "l_orderkey": 2945, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-01-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular instructions" }
+{ "l_orderkey": 3013, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18469.33, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-04-18", "l_receiptdate": "1997-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fluffily pending packages nag furiously al" }
+{ "l_orderkey": 3077, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24301.75, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-14", "l_commitdate": "1997-10-16", "l_receiptdate": "1997-10-06", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lent account" }
 { "l_orderkey": 3174, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37910.73, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1996-02-08", "l_receiptdate": "1995-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " wake slyly foxes. bold requests p" }
 { "l_orderkey": 3238, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ackages affix furiously. furiously bol" }
 { "l_orderkey": 3265, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6804.49, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "he forges. fluffily regular asym" }
+{ "l_orderkey": 3393, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46659.36, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-09-15", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " blithely final reques" }
+{ "l_orderkey": 3558, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16525.19, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-14", "l_commitdate": "1996-05-04", "l_receiptdate": "1996-04-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely unusual packa" }
 { "l_orderkey": 3655, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 34022.45, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-17", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ng foxes cajole fluffily slyly final fo" }
+{ "l_orderkey": 3969, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 38882.8, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "lar requests cajole furiously blithely regu" }
+{ "l_orderkey": 4005, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 27217.96, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1997-01-14", "l_receiptdate": "1996-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y pending dependenc" }
+{ "l_orderkey": 4196, "l_partkey": 72, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2916.21, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-07-21", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " accounts. fu" }
+{ "l_orderkey": 4672, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36938.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-28", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ests. idle, regular ex" }
 { "l_orderkey": 4742, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14581.05, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-20", "l_commitdate": "1995-05-26", "l_receiptdate": "1995-08-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "terns are sl" }
 { "l_orderkey": 4806, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5832.42, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-17", "l_commitdate": "1993-07-19", "l_receiptdate": "1993-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "even theodolites. packages sl" }
-{ "l_orderkey": 1606, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13594.98, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "structions haggle f" }
-{ "l_orderkey": 2183, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28161.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-08-24", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly unusual deposits sleep carefully" }
-{ "l_orderkey": 2407, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " wake carefully. fluffily " }
-{ "l_orderkey": 2497, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18450.33, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions? carefully daring accounts" }
-{ "l_orderkey": 3360, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 40784.94, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ages cajole. pending, " }
-{ "l_orderkey": 4295, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3884.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-05", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "arefully according to the pending ac" }
-{ "l_orderkey": 4772, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14566.05, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " requests. express, regular th" }
-{ "l_orderkey": 4967, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48553.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages. final, unusual accounts c" }
+{ "l_orderkey": 4898, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42771.08, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-13", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y regular grouches about" }
+{ "l_orderkey": 5313, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 29162.1, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-06-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding packages use" }
+{ "l_orderkey": 5443, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37910.73, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-01", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "gage carefully across the furiously" }
+{ "l_orderkey": 5540, "l_partkey": 72, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23329.68, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "deposits! ironic depths may engage-- b" }
+{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4860.35, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-02", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "yly along the ironic, fi" }
+{ "l_orderkey": 5635, "l_partkey": 72, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11664.84, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-11-17", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ke slyly against the carefully final req" }
 { "l_orderkey": 419, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14566.05, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-11-28", "l_receiptdate": "1996-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " sleep final, regular theodolites. fluffi" }
 { "l_orderkey": 518, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15537.12, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-15", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "use quickly expre" }
 { "l_orderkey": 547, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42727.08, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-08-17", "l_receiptdate": "1996-10-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "thely express dependencies. qu" }
-{ "l_orderkey": 673, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21363.54, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-04-27", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " the regular, even requests. carefully fin" }
-{ "l_orderkey": 1543, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33016.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ic requests are ac" }
-{ "l_orderkey": 4482, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-16", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-06-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " quickly pendin" }
-{ "l_orderkey": 4611, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46611.36, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
-{ "l_orderkey": 4839, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8739.63, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-17", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-07-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ounts haggle carefully above" }
-{ "l_orderkey": 4870, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34958.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " instructions. carefully pending pac" }
-{ "l_orderkey": 5952, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41756.01, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "posits sleep furiously quickly final p" }
 { "l_orderkey": 641, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24276.75, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-12-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "d, regular d" }
+{ "l_orderkey": 673, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21363.54, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-15", "l_commitdate": "1994-04-27", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " the regular, even requests. carefully fin" }
 { "l_orderkey": 899, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3884.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-11", "l_commitdate": "1998-05-14", "l_receiptdate": "1998-04-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ges. blithe, ironic waters cajole care" }
+{ "l_orderkey": 1543, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 33016.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ic requests are ac" }
+{ "l_orderkey": 1606, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13594.98, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-19", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "structions haggle f" }
 { "l_orderkey": 1635, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2913.21, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-13", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly ironic r" }
 { "l_orderkey": 1921, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 26218.89, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ing pinto beans above the pend" }
-{ "l_orderkey": 2146, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-10-19", "l_receiptdate": "1993-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y regular foxes wake among the final" }
-{ "l_orderkey": 2341, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35929.59, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-23", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "was blithel" }
-{ "l_orderkey": 3910, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30103.17, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-22", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess instructions. " }
-{ "l_orderkey": 4544, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19421.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-12", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " waters about the" }
-{ "l_orderkey": 4834, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25247.82, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ages dazzle carefully. slyly daring foxes" }
 { "l_orderkey": 2017, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13594.98, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-28", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-07-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ily final w" }
 { "l_orderkey": 2112, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-16", "l_receiptdate": "1997-05-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "lphins solve ideas. even, special reque" }
+{ "l_orderkey": 2146, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-10-19", "l_receiptdate": "1993-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y regular foxes wake among the final" }
+{ "l_orderkey": 2183, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 28161.03, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-21", "l_commitdate": "1996-08-24", "l_receiptdate": "1996-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly unusual deposits sleep carefully" }
 { "l_orderkey": 2212, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-22", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " cajole. final, pending ideas should are bl" }
+{ "l_orderkey": 2341, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35929.59, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-23", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-10-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "was blithel" }
+{ "l_orderkey": 2407, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17479.26, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-03", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " wake carefully. fluffily " }
+{ "l_orderkey": 2497, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18450.33, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-10", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " instructions? carefully daring accounts" }
 { "l_orderkey": 2884, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39813.87, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1997-12-17", "l_receiptdate": "1998-01-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep. slyly even accounts a" }
 { "l_orderkey": 3207, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 40784.94, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-06-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to the quickly special accounts? ironically" }
+{ "l_orderkey": 3360, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 40784.94, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ages cajole. pending, " }
+{ "l_orderkey": 3910, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 30103.17, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-22", "l_commitdate": "1996-11-14", "l_receiptdate": "1997-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ess instructions. " }
+{ "l_orderkey": 4295, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3884.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-05", "l_commitdate": "1996-04-26", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "arefully according to the pending ac" }
+{ "l_orderkey": 4482, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 31074.24, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-16", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-06-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " quickly pendin" }
+{ "l_orderkey": 4544, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19421.4, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-12", "l_commitdate": "1997-10-11", "l_receiptdate": "1997-10-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " waters about the" }
+{ "l_orderkey": 4611, "l_partkey": 71, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46611.36, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-12", "l_receiptdate": "1993-03-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular accounts " }
+{ "l_orderkey": 4772, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14566.05, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-09-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " requests. express, regular th" }
+{ "l_orderkey": 4834, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25247.82, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ages dazzle carefully. slyly daring foxes" }
+{ "l_orderkey": 4839, "l_partkey": 71, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8739.63, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-17", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-07-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ounts haggle carefully above" }
+{ "l_orderkey": 4870, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34958.52, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " instructions. carefully pending pac" }
+{ "l_orderkey": 4967, "l_partkey": 71, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48553.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-27", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kages. final, unusual accounts c" }
 { "l_orderkey": 5285, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 971.07, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-14", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e fluffily about the slyly special pa" }
+{ "l_orderkey": 5952, "l_partkey": 71, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41756.01, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-29", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "posits sleep furiously quickly final p" }
+{ "l_orderkey": 130, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 30072.17, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thily about the ruth" }
+{ "l_orderkey": 417, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "- final requests sle" }
+{ "l_orderkey": 645, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44623.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular dependencies across the speci" }
 { "l_orderkey": 1155, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3880.28, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic foxes according to the carefully final " }
+{ "l_orderkey": 1253, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21341.54, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "telets cajole alongside of the final reques" }
+{ "l_orderkey": 1569, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29102.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages. excuses lose evenly carefully reg" }
+{ "l_orderkey": 1984, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33952.45, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "tes. quickly pending packages haggle boldl" }
 { "l_orderkey": 2213, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 970.07, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-04-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s along the ironic reques" }
+{ "l_orderkey": 2629, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32012.31, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-14", "l_receiptdate": "1998-05-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. slowly express accounts are along the" }
+{ "l_orderkey": 2757, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13580.98, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "special deposits u" }
 { "l_orderkey": 2819, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16491.19, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-16", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-07-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "en deposits above the f" }
+{ "l_orderkey": 2949, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48503.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "gular courts cajole across t" }
 { "l_orderkey": 2977, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24251.75, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-21", "l_commitdate": "1996-10-06", "l_receiptdate": "1996-10-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "furiously pe" }
+{ "l_orderkey": 2982, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20371.47, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-06-03", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular ideas use furiously? bl" }
+{ "l_orderkey": 3079, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ets are according to the quickly dari" }
+{ "l_orderkey": 3205, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "symptotes. slyly even deposits ar" }
 { "l_orderkey": 3587, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11640.84, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-30", "l_commitdate": "1996-07-04", "l_receiptdate": "1996-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "g the even pinto beans. special," }
 { "l_orderkey": 3649, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13580.98, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ithely bold accounts wake " }
 { "l_orderkey": 3650, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 43.0, "l_extendedprice": 41713.01, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "structions use caref" }
 { "l_orderkey": 3718, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7760.56, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-06", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " the even deposits sleep carefully b" }
-{ "l_orderkey": 4455, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1993-11-21", "l_receiptdate": "1994-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express packages. packages boost quickly" }
-{ "l_orderkey": 5124, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34922.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r deposits ab" }
-{ "l_orderkey": 5636, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-05-17", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "slyly express requests. furiously pen" }
-{ "l_orderkey": 417, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-29", "l_commitdate": "1994-04-10", "l_receiptdate": "1994-04-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "- final requests sle" }
-{ "l_orderkey": 645, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44623.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-04", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " regular dependencies across the speci" }
-{ "l_orderkey": 1253, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21341.54, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-23", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "telets cajole alongside of the final reques" }
-{ "l_orderkey": 1984, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33952.45, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-18", "l_commitdate": "1998-05-04", "l_receiptdate": "1998-06-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "tes. quickly pending packages haggle boldl" }
-{ "l_orderkey": 2629, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 32012.31, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-29", "l_commitdate": "1998-05-14", "l_receiptdate": "1998-05-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. slowly express accounts are along the" }
-{ "l_orderkey": 2949, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48503.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-04", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "gular courts cajole across t" }
-{ "l_orderkey": 2982, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20371.47, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-06-03", "l_receiptdate": "1995-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "egular ideas use furiously? bl" }
-{ "l_orderkey": 3079, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-10-26", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ets are according to the quickly dari" }
-{ "l_orderkey": 5473, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26191.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the deposits. warthogs wake fur" }
-{ "l_orderkey": 2757, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13580.98, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "special deposits u" }
 { "l_orderkey": 3872, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 40742.94, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-12", "l_receiptdate": "1997-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s the furio" }
 { "l_orderkey": 3937, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46563.36, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-15", "l_commitdate": "1998-02-22", "l_receiptdate": "1998-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gainst the thinl" }
+{ "l_orderkey": 4197, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ronic requests. quickly bold packages in" }
 { "l_orderkey": 4199, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15521.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-07-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ncies. furiously special accounts" }
+{ "l_orderkey": 4455, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19401.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1993-11-21", "l_receiptdate": "1994-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express packages. packages boost quickly" }
+{ "l_orderkey": 4513, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly furiously unusual deposits. blit" }
 { "l_orderkey": 4992, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24251.75, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-06", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-08-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly about the never ironic requests. pe" }
 { "l_orderkey": 5059, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4850.35, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-23", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ts affix slyly accordi" }
+{ "l_orderkey": 5124, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34922.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-20", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "r deposits ab" }
+{ "l_orderkey": 5473, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26191.89, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the deposits. warthogs wake fur" }
 { "l_orderkey": 5605, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-13", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "cial deposits. theodolites w" }
-{ "l_orderkey": 5984, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12610.91, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-06", "l_receiptdate": "1994-11-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar platelets. f" }
-{ "l_orderkey": 130, "l_partkey": 70, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 30072.17, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-09-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "thily about the ruth" }
-{ "l_orderkey": 1569, "l_partkey": 70, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 29102.1, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "packages. excuses lose evenly carefully reg" }
-{ "l_orderkey": 3205, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-08-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "symptotes. slyly even deposits ar" }
-{ "l_orderkey": 4197, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-11", "l_receiptdate": "1996-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ronic requests. quickly bold packages in" }
-{ "l_orderkey": 4513, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37832.73, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-25", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-07-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly furiously unusual deposits. blit" }
+{ "l_orderkey": 5636, "l_partkey": 70, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17461.26, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-05-17", "l_receiptdate": "1995-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "slyly express requests. furiously pen" }
 { "l_orderkey": 5636, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25221.82, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-05", "l_commitdate": "1995-05-16", "l_receiptdate": "1995-03-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously final pinto beans o" }
+{ "l_orderkey": 5984, "l_partkey": 70, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12610.91, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-06", "l_receiptdate": "1994-11-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar platelets. f" }
+{ "l_orderkey": 386, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15504.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely fluffi" }
+{ "l_orderkey": 1025, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22288.38, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular platelets nag carefu" }
 { "l_orderkey": 1221, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2907.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ing to the fluffily" }
+{ "l_orderkey": 1477, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 47483.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-18", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ise according to the sly, bold p" }
 { "l_orderkey": 1600, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24226.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "press packages. ironic excuses bo" }
 { "l_orderkey": 1824, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38762.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-24", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es mold furiously final instructions. s" }
+{ "l_orderkey": 2500, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16474.02, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "encies-- ironic, even packages" }
+{ "l_orderkey": 2849, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29071.8, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-07-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly furiously even id" }
+{ "l_orderkey": 3045, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46514.88, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole quickly outside th" }
+{ "l_orderkey": 3205, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 34886.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. ironic platelets above the s" }
+{ "l_orderkey": 3429, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ackages. quickly e" }
+{ "l_orderkey": 3459, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously silent dolphi" }
+{ "l_orderkey": 3591, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23257.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages. slyly regular dependencies cajo" }
+{ "l_orderkey": 3714, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12597.78, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the furiously final" }
 { "l_orderkey": 3717, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4845.3, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-02", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-09-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "quickly among " }
+{ "l_orderkey": 3940, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38762.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-03-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. regular fox" }
 { "l_orderkey": 4102, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4845.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-05-11", "l_receiptdate": "1996-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " the furiously even" }
+{ "l_orderkey": 4322, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37793.34, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its integrate fluffily " }
+{ "l_orderkey": 4577, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11628.72, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests alongsi" }
+{ "l_orderkey": 4737, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21319.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " hang fluffily around t" }
 { "l_orderkey": 4769, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 43607.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-01", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "accounts are. even accounts sleep" }
 { "l_orderkey": 5216, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16474.02, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s according to the accounts bo" }
 { "l_orderkey": 5348, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20350.26, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-11", "l_commitdate": "1997-12-24", "l_receiptdate": "1997-12-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular theodolites haggle car" }
-{ "l_orderkey": 1025, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22288.38, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " regular platelets nag carefu" }
-{ "l_orderkey": 3940, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38762.4, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-03-04", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. regular fox" }
-{ "l_orderkey": 4577, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11628.72, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests alongsi" }
-{ "l_orderkey": 4737, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21319.32, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-05-22", "l_receiptdate": "1993-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " hang fluffily around t" }
-{ "l_orderkey": 3045, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46514.88, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole quickly outside th" }
-{ "l_orderkey": 3429, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ackages. quickly e" }
-{ "l_orderkey": 3459, "l_partkey": 69, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9690.6, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-06", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously silent dolphi" }
-{ "l_orderkey": 386, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15504.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-12", "l_commitdate": "1995-04-18", "l_receiptdate": "1995-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely fluffi" }
-{ "l_orderkey": 1477, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 49.0, "l_extendedprice": 47483.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-18", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ise according to the sly, bold p" }
-{ "l_orderkey": 2500, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16474.02, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "encies-- ironic, even packages" }
-{ "l_orderkey": 2849, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 29071.8, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-20", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-07-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly furiously even id" }
-{ "l_orderkey": 3205, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 34886.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. ironic platelets above the s" }
-{ "l_orderkey": 3591, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 23257.44, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-26", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages. slyly regular dependencies cajo" }
-{ "l_orderkey": 3714, "l_partkey": 69, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12597.78, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the furiously final" }
-{ "l_orderkey": 4322, "l_partkey": 69, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37793.34, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "its integrate fluffily " }
 { "l_orderkey": 5767, "l_partkey": 69, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14535.9, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warthogs. carefully unusual g" }
-{ "l_orderkey": 1507, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24201.5, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-06", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xes. slyly busy de" }
-{ "l_orderkey": 1634, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cial, bold platelets alongside of the f" }
-{ "l_orderkey": 1890, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 41626.58, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly. instructions across the furiously" }
-{ "l_orderkey": 1927, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts affi" }
-{ "l_orderkey": 2052, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48403.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-07-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "wake after the decoy" }
-{ "l_orderkey": 3205, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-17", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly alongsi" }
-{ "l_orderkey": 3873, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final ac" }
-{ "l_orderkey": 5121, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e quickly according " }
-{ "l_orderkey": 5414, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 38722.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts are evenly across" }
-{ "l_orderkey": 5602, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29041.8, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e slyly even packages. careful" }
-{ "l_orderkey": 5664, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 32914.04, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-10-05", "l_receiptdate": "1998-09-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "st. fluffily pending foxes na" }
 { "l_orderkey": 1, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34850.16, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly final dependencies: slyly bold " }
-{ "l_orderkey": 868, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironic platelets wake. rut" }
-{ "l_orderkey": 2465, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26137.62, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits boost carefully unusual instructio" }
-{ "l_orderkey": 4706, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5808.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully eve" }
-{ "l_orderkey": 4865, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42594.64, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even deposits sleep against the quickly r" }
-{ "l_orderkey": 5472, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27105.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ffily pendin" }
-{ "l_orderkey": 5921, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-05-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "final asymptotes. even packages boost " }
 { "l_orderkey": 39, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44530.76, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-26", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he carefully e" }
 { "l_orderkey": 135, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32914.04, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1996-01-19", "l_receiptdate": "1996-02-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "counts doze against the blithely ironi" }
-{ "l_orderkey": 1473, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30977.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the packages lose furiously ab" }
-{ "l_orderkey": 1543, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quickly. final accounts haggle slyl" }
-{ "l_orderkey": 1826, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-12", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " blithely special" }
-{ "l_orderkey": 1926, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12584.78, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-26", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "eans wake bli" }
-{ "l_orderkey": 2438, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-09-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "engage car" }
-{ "l_orderkey": 3041, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "scapades after the special" }
-{ "l_orderkey": 4837, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40658.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "o the furiously final theodolites boost" }
-{ "l_orderkey": 5252, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23233.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-17", "l_receiptdate": "1996-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "posits after the fluffi" }
-{ "l_orderkey": 5504, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3872.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "into beans boost. " }
 { "l_orderkey": 486, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 38722.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-06-06", "l_receiptdate": "1996-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ts nag quickly among the slyl" }
 { "l_orderkey": 610, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10648.66, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-31", "l_commitdate": "1995-10-25", "l_receiptdate": "1995-11-18", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely final " }
+{ "l_orderkey": 868, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lyly ironic platelets wake. rut" }
+{ "l_orderkey": 1473, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30977.92, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the packages lose furiously ab" }
+{ "l_orderkey": 1507, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24201.5, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-07", "l_commitdate": "1994-01-06", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xes. slyly busy de" }
 { "l_orderkey": 1539, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ". fluffily reg" }
+{ "l_orderkey": 1543, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-22", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quickly. final accounts haggle slyl" }
+{ "l_orderkey": 1634, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-29", "l_commitdate": "1996-10-15", "l_receiptdate": "1996-11-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cial, bold platelets alongside of the f" }
+{ "l_orderkey": 1826, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-12", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " blithely special" }
 { "l_orderkey": 1861, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-14", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s foxes. slyly" }
+{ "l_orderkey": 1890, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 41626.58, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-08", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lyly. instructions across the furiously" }
+{ "l_orderkey": 1926, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12584.78, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-26", "l_commitdate": "1996-04-13", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "eans wake bli" }
+{ "l_orderkey": 1927, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2904.18, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ccounts affi" }
+{ "l_orderkey": 2052, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 48403.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-22", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-07-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "wake after the decoy" }
+{ "l_orderkey": 2438, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-09-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "engage car" }
+{ "l_orderkey": 2465, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26137.62, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits boost carefully unusual instructio" }
+{ "l_orderkey": 3041, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8712.54, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "scapades after the special" }
+{ "l_orderkey": 3205, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6776.42, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-17", "l_receiptdate": "1992-07-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly alongsi" }
 { "l_orderkey": 3842, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12584.78, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-13", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "t blithely. busily regular accounts alon" }
+{ "l_orderkey": 3873, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18393.14, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-15", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final ac" }
+{ "l_orderkey": 4706, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5808.36, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-03-18", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully eve" }
+{ "l_orderkey": 4837, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40658.52, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-19", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "o the furiously final theodolites boost" }
+{ "l_orderkey": 4865, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42594.64, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-25", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-08-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "even deposits sleep against the quickly r" }
+{ "l_orderkey": 5121, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9680.6, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-08", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e quickly according " }
 { "l_orderkey": 5153, "l_partkey": 68, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29041.8, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-14", "l_receiptdate": "1995-11-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "beans sleep bl" }
+{ "l_orderkey": 5252, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23233.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-11", "l_commitdate": "1996-04-17", "l_receiptdate": "1996-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "posits after the fluffi" }
+{ "l_orderkey": 5414, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 38722.4, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-07", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ts are evenly across" }
+{ "l_orderkey": 5472, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27105.68, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-05-28", "l_receiptdate": "1993-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ffily pendin" }
+{ "l_orderkey": 5504, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3872.24, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "into beans boost. " }
+{ "l_orderkey": 5602, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 29041.8, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "e slyly even packages. careful" }
+{ "l_orderkey": 5664, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 32914.04, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-10", "l_commitdate": "1998-10-05", "l_receiptdate": "1998-09-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "st. fluffily pending foxes na" }
 { "l_orderkey": 5857, "l_partkey": 68, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 968.06, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1997-12-09", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "instructions detect final reques" }
+{ "l_orderkey": 5921, "l_partkey": 68, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16457.02, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-05-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "final asymptotes. even packages boost " }
+{ "l_orderkey": 612, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 47385.94, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1992-11-25", "l_receiptdate": "1993-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "theodolite" }
 { "l_orderkey": 1445, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46418.88, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". final ideas are carefully dar" }
-{ "l_orderkey": 1702, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ies haggle blith" }
-{ "l_orderkey": 3136, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1934.12, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "? special, silent " }
-{ "l_orderkey": 5344, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25143.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-27", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending, silent multipliers." }
-{ "l_orderkey": 5507, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21275.32, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gular ideas. carefully unu" }
 { "l_orderkey": 1543, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40616.52, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "its sleep until the fur" }
+{ "l_orderkey": 1702, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-02", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-06-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ies haggle blith" }
 { "l_orderkey": 1764, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "es wake slowly. " }
 { "l_orderkey": 2049, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17407.08, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-09", "l_commitdate": "1996-01-22", "l_receiptdate": "1996-01-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " sleep fluffily. dependencies use never" }
-{ "l_orderkey": 2819, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11604.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-18", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-07-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " regular, regular a" }
-{ "l_orderkey": 3426, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "c accounts cajole carefu" }
-{ "l_orderkey": 4102, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37715.34, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ffix blithely slyly special " }
-{ "l_orderkey": 5441, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45451.82, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-16", "l_receiptdate": "1994-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ounts wake slyly about the express instr" }
-{ "l_orderkey": 5734, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9670.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1997-12-24", "l_receiptdate": "1998-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests; accounts above" }
-{ "l_orderkey": 612, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 47385.94, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1992-11-25", "l_receiptdate": "1993-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "theodolite" }
 { "l_orderkey": 2631, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3868.24, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-03", "l_commitdate": "1993-12-17", "l_receiptdate": "1993-11-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "special theodolites. a" }
 { "l_orderkey": 2661, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10637.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-14", "l_commitdate": "1997-02-11", "l_receiptdate": "1997-05-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "equests are a" }
+{ "l_orderkey": 2819, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11604.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-18", "l_commitdate": "1994-06-24", "l_receiptdate": "1994-07-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " regular, regular a" }
+{ "l_orderkey": 3136, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1934.12, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-03", "l_receiptdate": "1994-11-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "? special, silent " }
 { "l_orderkey": 3399, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "hely pending dugouts " }
+{ "l_orderkey": 3426, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18374.14, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-07", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "c accounts cajole carefu" }
+{ "l_orderkey": 4102, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37715.34, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-14", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ffix blithely slyly special " }
 { "l_orderkey": 4929, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 23209.44, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-04-30", "l_receiptdate": "1996-05-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts boost" }
 { "l_orderkey": 5317, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48353.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-17", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-11-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "cajole furiously. accounts use quick" }
+{ "l_orderkey": 5344, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 25143.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-27", "l_commitdate": "1998-08-22", "l_receiptdate": "1998-09-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending, silent multipliers." }
+{ "l_orderkey": 5441, "l_partkey": 67, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45451.82, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-19", "l_commitdate": "1994-10-16", "l_receiptdate": "1994-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ounts wake slyly about the express instr" }
+{ "l_orderkey": 5507, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 21275.32, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "gular ideas. carefully unu" }
 { "l_orderkey": 5543, "l_partkey": 67, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2901.18, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-11-05", "l_receiptdate": "1993-12-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress, even " }
 { "l_orderkey": 5632, "l_partkey": 67, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23209.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-23", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "beans detect. quickly final i" }
+{ "l_orderkey": 5734, "l_partkey": 67, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9670.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1997-12-24", "l_receiptdate": "1998-01-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "equests; accounts above" }
+{ "l_orderkey": 71, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2898.18, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-23", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. pinto beans haggle after the" }
 { "l_orderkey": 194, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12558.78, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-07", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-05-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "about the blit" }
+{ "l_orderkey": 261, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ites hinder " }
+{ "l_orderkey": 549, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34778.16, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-10-11", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts against the ironic, even theodolites eng" }
 { "l_orderkey": 609, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20287.26, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "de of the special warthogs. excu" }
-{ "l_orderkey": 1280, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8694.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usual accou" }
-{ "l_orderkey": 1603, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28015.74, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-10-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ses wake furiously. theodolite" }
-{ "l_orderkey": 3015, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests wake fluffil" }
 { "l_orderkey": 740, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33812.1, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-08-22", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "p quickly. fu" }
 { "l_orderkey": 995, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-08", "l_commitdate": "1995-08-05", "l_receiptdate": "1995-09-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lyly even " }
+{ "l_orderkey": 1280, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8694.54, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "usual accou" }
+{ "l_orderkey": 1316, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14490.9, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1993-12-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully express dugouts. furiously silent ide" }
+{ "l_orderkey": 1410, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-05-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts haggle against the furiously fina" }
+{ "l_orderkey": 1603, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 28015.74, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-28", "l_commitdate": "1993-09-20", "l_receiptdate": "1993-10-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ses wake furiously. theodolite" }
 { "l_orderkey": 1733, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8694.54, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-07-23", "l_receiptdate": "1996-06-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ven foxes was according to t" }
+{ "l_orderkey": 2343, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33812.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ges haggle furiously carefully regular req" }
 { "l_orderkey": 2562, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " accounts-- silent, unusual ideas a" }
+{ "l_orderkey": 2592, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1932.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-24", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "side of the b" }
+{ "l_orderkey": 2887, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10626.66, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-17", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. unusual, speci" }
+{ "l_orderkey": 2914, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully about the fluffily ironic gifts" }
+{ "l_orderkey": 2950, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uests cajole furio" }
+{ "l_orderkey": 3015, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-10", "l_commitdate": "1992-11-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests wake fluffil" }
+{ "l_orderkey": 3143, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44438.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "low forges haggle. even packages use bli" }
+{ "l_orderkey": 3814, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". doggedly ironic deposits will have to wa" }
 { "l_orderkey": 4193, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 20287.26, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-03-22", "l_receiptdate": "1994-05-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "accounts cajole b" }
+{ "l_orderkey": 4195, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly express pinto bea" }
+{ "l_orderkey": 4550, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18355.14, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. express " }
+{ "l_orderkey": 4645, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30913.92, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final accounts alongside" }
 { "l_orderkey": 5380, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5796.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1998-01-08", "l_receiptdate": "1997-12-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "es. fluffily brave accounts across t" }
 { "l_orderkey": 5412, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 46370.88, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-22", "l_commitdate": "1998-03-28", "l_receiptdate": "1998-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s. slyly final packages cajole blithe" }
 { "l_orderkey": 5637, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15456.96, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-08-31", "l_receiptdate": "1996-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "d packages. express requests" }
 { "l_orderkey": 5922, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12558.78, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sly special accounts wake ironically." }
-{ "l_orderkey": 71, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2898.18, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-23", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y. pinto beans haggle after the" }
-{ "l_orderkey": 549, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34778.16, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-10-11", "l_receiptdate": "1992-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ts against the ironic, even theodolites eng" }
-{ "l_orderkey": 1410, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 24151.5, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-05-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts haggle against the furiously fina" }
-{ "l_orderkey": 2343, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33812.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ges haggle furiously carefully regular req" }
-{ "l_orderkey": 2887, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10626.66, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-17", "l_receiptdate": "1997-07-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. unusual, speci" }
-{ "l_orderkey": 2914, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " carefully about the fluffily ironic gifts" }
-{ "l_orderkey": 2950, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17389.08, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-29", "l_receiptdate": "1997-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uests cajole furio" }
-{ "l_orderkey": 3143, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 44438.76, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-19", "l_commitdate": "1993-03-21", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "low forges haggle. even packages use bli" }
-{ "l_orderkey": 4195, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 21253.32, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-01", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-07-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lly express pinto bea" }
-{ "l_orderkey": 261, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-02", "l_receiptdate": "1993-11-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ites hinder " }
-{ "l_orderkey": 1316, "l_partkey": 66, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14490.9, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1994-02-04", "l_receiptdate": "1993-12-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully express dugouts. furiously silent ide" }
-{ "l_orderkey": 2592, "l_partkey": 66, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1932.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-24", "l_commitdate": "1993-04-05", "l_receiptdate": "1993-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "side of the b" }
-{ "l_orderkey": 3814, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19321.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": ". doggedly ironic deposits will have to wa" }
-{ "l_orderkey": 4550, "l_partkey": 66, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18355.14, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-01", "l_commitdate": "1995-02-13", "l_receiptdate": "1995-01-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests. express " }
-{ "l_orderkey": 4645, "l_partkey": 66, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30913.92, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-10-30", "l_receiptdate": "1994-11-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final accounts alongside" }
-{ "l_orderkey": 166, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar frays wake blithely a" }
-{ "l_orderkey": 419, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30881.92, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely regular requests. special pinto" }
-{ "l_orderkey": 903, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26056.62, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-09-20", "l_receiptdate": "1995-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lly pending foxes. furiously" }
-{ "l_orderkey": 1347, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8685.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-09-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " detect blithely above the fina" }
-{ "l_orderkey": 1927, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5790.36, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "furiously even wat" }
-{ "l_orderkey": 2503, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27021.68, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake quickly slyly " }
-{ "l_orderkey": 2528, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-25", "l_commitdate": "1995-02-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng the pending excuses haggle after the bl" }
-{ "l_orderkey": 2848, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42462.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ions. slyly express instructions n" }
-{ "l_orderkey": 4354, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-29", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "deas use blithely! special foxes print af" }
-{ "l_orderkey": 4804, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", thin excuses. " }
-{ "l_orderkey": 4995, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15440.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular, bold packages. accou" }
 { "l_orderkey": 70, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7720.48, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-12", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ggle. carefully pending dependenc" }
-{ "l_orderkey": 930, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9650.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckly regular requests: regular instructions" }
-{ "l_orderkey": 3751, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43427.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "according to " }
-{ "l_orderkey": 4614, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-08-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ions engage final, ironic " }
-{ "l_orderkey": 4711, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7720.48, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g to the carefully ironic deposits. specia" }
-{ "l_orderkey": 4805, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12545.78, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its serve about the accounts. slyly regu" }
-{ "l_orderkey": 5095, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44392.76, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular instruction" }
+{ "l_orderkey": 166, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-16", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar frays wake blithely a" }
 { "l_orderkey": 388, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 38602.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1993-01-28", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "quests against the carefully unusual epi" }
+{ "l_orderkey": 419, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30881.92, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "blithely regular requests. special pinto" }
 { "l_orderkey": 512, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5790.36, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-06-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en ideas haggle " }
-{ "l_orderkey": 2785, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fter the furiously final p" }
-{ "l_orderkey": 3430, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48253.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-03-03", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic theodolites. carefully regular pac" }
-{ "l_orderkey": 3553, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25091.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fily special p" }
-{ "l_orderkey": 4257, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-18", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thin the theodolites use after the bl" }
-{ "l_orderkey": 4388, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28951.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s cajole fluffil" }
-{ "l_orderkey": 5539, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40532.52, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ons across the carefully si" }
+{ "l_orderkey": 903, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26056.62, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-09-20", "l_receiptdate": "1995-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lly pending foxes. furiously" }
+{ "l_orderkey": 930, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9650.6, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckly regular requests: regular instructions" }
 { "l_orderkey": 935, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 22196.38, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-11", "l_commitdate": "1997-11-25", "l_receiptdate": "1998-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "hes haggle furiously dolphins. qu" }
 { "l_orderkey": 994, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3860.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-05-21", "l_receiptdate": "1994-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "aggle carefully acc" }
 { "l_orderkey": 1030, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16406.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-13", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly. carefully even packages dazz" }
+{ "l_orderkey": 1347, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8685.54, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-09-16", "l_receiptdate": "1997-09-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " detect blithely above the fina" }
 { "l_orderkey": 1409, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34742.16, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ncies sleep carefully r" }
+{ "l_orderkey": 1927, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5790.36, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "furiously even wat" }
 { "l_orderkey": 2053, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 44392.76, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-04-02", "l_receiptdate": "1995-04-18", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tions. furiously even requests hagg" }
+{ "l_orderkey": 2503, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 27021.68, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-08", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-08-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s wake quickly slyly " }
+{ "l_orderkey": 2528, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-25", "l_commitdate": "1995-02-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ng the pending excuses haggle after the bl" }
 { "l_orderkey": 2563, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9650.6, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1993-12-19", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tealthily abo" }
+{ "l_orderkey": 2785, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-24", "l_receiptdate": "1995-11-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fter the furiously final p" }
+{ "l_orderkey": 2848, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42462.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-14", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ions. slyly express instructions n" }
 { "l_orderkey": 3106, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 15440.96, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-03-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "sits wake slyl" }
 { "l_orderkey": 3270, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 19301.2, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-07-31", "l_receiptdate": "1997-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "en accounts among the c" }
+{ "l_orderkey": 3430, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 48253.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1995-03-03", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ironic theodolites. carefully regular pac" }
+{ "l_orderkey": 3553, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 25091.56, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-06", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "fily special p" }
+{ "l_orderkey": 3751, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 43427.7, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-06-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "according to " }
+{ "l_orderkey": 4257, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-18", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thin the theodolites use after the bl" }
+{ "l_orderkey": 4354, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35707.22, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-29", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "deas use blithely! special foxes print af" }
+{ "l_orderkey": 4388, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28951.8, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-07", "l_commitdate": "1996-05-07", "l_receiptdate": "1996-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s cajole fluffil" }
+{ "l_orderkey": 4614, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2895.18, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-07-21", "l_receiptdate": "1996-08-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ions engage final, ironic " }
+{ "l_orderkey": 4711, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7720.48, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-06-13", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g to the carefully ironic deposits. specia" }
+{ "l_orderkey": 4804, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31846.98, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": ", thin excuses. " }
+{ "l_orderkey": 4805, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12545.78, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-08-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its serve about the accounts. slyly regu" }
 { "l_orderkey": 4865, "l_partkey": 65, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 45357.82, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-08-07", "l_receiptdate": "1997-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y unusual packages. packages" }
+{ "l_orderkey": 4995, "l_partkey": 65, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15440.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-27", "l_commitdate": "1996-04-03", "l_receiptdate": "1996-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "egular, bold packages. accou" }
+{ "l_orderkey": 5095, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44392.76, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-26", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-07-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular instruction" }
 { "l_orderkey": 5376, "l_partkey": 65, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17371.08, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-09-13", "l_receiptdate": "1994-11-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts boo" }
+{ "l_orderkey": 5539, "l_partkey": 65, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40532.52, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-17", "l_receiptdate": "1994-10-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ons across the carefully si" }
 { "l_orderkey": 1, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7712.48, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously. regular, express dep" }
-{ "l_orderkey": 581, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts. quickly" }
-{ "l_orderkey": 2470, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9640.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic requests a" }
-{ "l_orderkey": 3266, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "grate among the quickly express deposits" }
-{ "l_orderkey": 4037, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30849.92, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e of the pending, iron" }
-{ "l_orderkey": 4039, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "t? pinto beans cajole across the thinly r" }
-{ "l_orderkey": 4832, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5784.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-02-03", "l_receiptdate": "1997-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ages cajole after the bold requests. furi" }
-{ "l_orderkey": 5409, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-26", "l_receiptdate": "1992-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "osits cajole furiously" }
-{ "l_orderkey": 5540, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18317.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1996-11-18", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slyly slyl" }
 { "l_orderkey": 352, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16389.02, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending deposits sleep furiously " }
-{ "l_orderkey": 2407, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-10", "l_commitdate": "1998-08-25", "l_receiptdate": "1998-10-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "l dependencies s" }
-{ "l_orderkey": 3653, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 27957.74, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ording to the special, final" }
-{ "l_orderkey": 3717, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 36634.28, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly about the car" }
-{ "l_orderkey": 4004, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 45310.82, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-08-03", "l_receiptdate": "1993-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "thely instead of the even, unu" }
-{ "l_orderkey": 4704, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-02", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the care" }
 { "l_orderkey": 384, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 47238.94, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-04-25", "l_receiptdate": "1992-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "refully carefully ironic instructions. bl" }
-{ "l_orderkey": 3172, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". slyly regular dependencies haggle quiet" }
-{ "l_orderkey": 3331, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "odolites. bold accounts" }
-{ "l_orderkey": 3520, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully pendi" }
-{ "l_orderkey": 5826, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17353.08, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-17", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-07-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "atelets use above t" }
 { "l_orderkey": 577, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "l accounts wake deposits. ironic packa" }
+{ "l_orderkey": 581, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nts. quickly" }
 { "l_orderkey": 1541, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-07-13", "l_receiptdate": "1995-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "o beans boost fluffily abou" }
 { "l_orderkey": 1666, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19281.2, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-12-12", "l_receiptdate": "1996-01-31", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uietly regular foxes wake quick" }
 { "l_orderkey": 2209, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10604.66, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-12", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-08-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "express, regular pinto be" }
 { "l_orderkey": 2213, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2892.18, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-09", "l_commitdate": "1993-03-17", "l_receiptdate": "1993-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "o wake. ironic platel" }
+{ "l_orderkey": 2407, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-10", "l_commitdate": "1998-08-25", "l_receiptdate": "1998-10-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "l dependencies s" }
+{ "l_orderkey": 2470, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9640.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " ironic requests a" }
 { "l_orderkey": 2628, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40490.52, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-20", "l_commitdate": "1994-01-04", "l_receiptdate": "1993-12-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ld notornis alongside " }
 { "l_orderkey": 2755, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 20245.26, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-13", "l_commitdate": "1992-04-20", "l_receiptdate": "1992-03-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furious re" }
+{ "l_orderkey": 3172, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": ". slyly regular dependencies haggle quiet" }
+{ "l_orderkey": 3266, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29885.86, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "grate among the quickly express deposits" }
 { "l_orderkey": 3271, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27957.74, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-10", "l_commitdate": "1992-02-05", "l_receiptdate": "1992-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lar instructions. carefully regular" }
+{ "l_orderkey": 3331, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-07-03", "l_receiptdate": "1993-08-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "odolites. bold accounts" }
+{ "l_orderkey": 3520, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 39526.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-06", "l_commitdate": "1997-09-20", "l_receiptdate": "1997-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully pendi" }
+{ "l_orderkey": 3653, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 27957.74, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ording to the special, final" }
 { "l_orderkey": 3712, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-26", "l_commitdate": "1992-02-19", "l_receiptdate": "1992-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ously permanently regular req" }
+{ "l_orderkey": 3717, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 36634.28, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly about the car" }
+{ "l_orderkey": 4004, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 45310.82, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-08-03", "l_receiptdate": "1993-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "thely instead of the even, unu" }
+{ "l_orderkey": 4037, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30849.92, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-06-08", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e of the pending, iron" }
+{ "l_orderkey": 4039, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8676.54, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-08", "l_commitdate": "1998-02-05", "l_receiptdate": "1998-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "t? pinto beans cajole across the thinly r" }
 { "l_orderkey": 4545, "l_partkey": 64, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1928.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-05-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ages use. slyly even i" }
 { "l_orderkey": 4676, "l_partkey": 64, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 12532.78, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-18", "l_commitdate": "1995-11-07", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " at the slyly bold attainments. silently e" }
+{ "l_orderkey": 4704, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 42418.64, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-02", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-09-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "out the care" }
+{ "l_orderkey": 4832, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5784.36, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-08", "l_commitdate": "1998-02-03", "l_receiptdate": "1997-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ages cajole after the bold requests. furi" }
+{ "l_orderkey": 5409, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13496.84, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-26", "l_receiptdate": "1992-02-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "osits cajole furiously" }
+{ "l_orderkey": 5540, "l_partkey": 64, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 18317.14, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-06", "l_commitdate": "1996-11-18", "l_receiptdate": "1997-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " slyly slyl" }
+{ "l_orderkey": 5826, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17353.08, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-17", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-07-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "atelets use above t" }
 { "l_orderkey": 3, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 25039.56, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges sleep after the caref" }
 { "l_orderkey": 100, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26965.68, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-13", "l_receiptdate": "1998-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sts haggle. slowl" }
+{ "l_orderkey": 869, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-02-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "uffily even excuses? slyly even deposits " }
 { "l_orderkey": 1636, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-09-09", "l_receiptdate": "1997-08-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ular depos" }
+{ "l_orderkey": 1863, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46226.88, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ans hinder furiou" }
+{ "l_orderkey": 1986, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13482.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the packages. pending, unusual" }
+{ "l_orderkey": 2016, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14445.9, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-24", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests haggle carefully furiously regul" }
 { "l_orderkey": 2624, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14445.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "le. quickly pending requests" }
 { "l_orderkey": 2791, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3852.24, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-02", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-01-29", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "slyly bold packages boost. slyly" }
-{ "l_orderkey": 3650, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29854.86, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " against the ironic accounts cajol" }
-{ "l_orderkey": 4130, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uriously regular instructions around th" }
-{ "l_orderkey": 4451, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " regular ideas." }
-{ "l_orderkey": 4998, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45263.82, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-07", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "mong the careful" }
-{ "l_orderkey": 1863, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46226.88, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-10", "l_commitdate": "1993-12-09", "l_receiptdate": "1993-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ans hinder furiou" }
-{ "l_orderkey": 2016, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14445.9, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-24", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests haggle carefully furiously regul" }
+{ "l_orderkey": 2886, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-18", "l_commitdate": "1995-01-31", "l_receiptdate": "1994-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ar theodolites. e" }
 { "l_orderkey": 3104, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10593.66, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-05", "l_commitdate": "1993-11-30", "l_receiptdate": "1993-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " special deposits u" }
+{ "l_orderkey": 3264, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5778.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "press packages. ironical" }
 { "l_orderkey": 3460, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 44300.76, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-02-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uses run among the carefully even deposits" }
 { "l_orderkey": 3461, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-10", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-03-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ely unusual deposits. quickly ir" }
+{ "l_orderkey": 3618, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23113.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-26", "l_commitdate": "1998-01-15", "l_receiptdate": "1998-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "xpress acc" }
+{ "l_orderkey": 3650, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29854.86, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-01", "l_receiptdate": "1992-07-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " against the ironic accounts cajol" }
+{ "l_orderkey": 4130, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-19", "l_commitdate": "1996-04-24", "l_receiptdate": "1996-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "uriously regular instructions around th" }
 { "l_orderkey": 4160, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 46226.88, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-19", "l_commitdate": "1996-11-02", "l_receiptdate": "1996-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " unusual dolphins " }
+{ "l_orderkey": 4451, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " regular ideas." }
 { "l_orderkey": 4545, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-07", "l_commitdate": "1993-02-18", "l_receiptdate": "1993-02-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously bold asymptotes! blithely pen" }
 { "l_orderkey": 4769, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32744.04, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-26", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ven instructions. ca" }
-{ "l_orderkey": 5381, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47189.94, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-06-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " accounts. regular, regula" }
-{ "l_orderkey": 869, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 26002.62, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-30", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-02-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "uffily even excuses? slyly even deposits " }
-{ "l_orderkey": 1986, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13482.84, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-14", "l_commitdate": "1994-06-19", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "the packages. pending, unusual" }
-{ "l_orderkey": 3264, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5778.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-10", "l_commitdate": "1996-12-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "press packages. ironical" }
-{ "l_orderkey": 3618, "l_partkey": 63, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23113.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-26", "l_commitdate": "1998-01-15", "l_receiptdate": "1998-02-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "xpress acc" }
 { "l_orderkey": 4871, "l_partkey": 63, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2889.18, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-08-10", "l_receiptdate": "1995-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y special packages wak" }
-{ "l_orderkey": 2886, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1926.12, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-18", "l_commitdate": "1995-01-31", "l_receiptdate": "1994-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ar theodolites. e" }
+{ "l_orderkey": 4998, "l_partkey": 63, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 45263.82, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-07", "l_commitdate": "1992-03-07", "l_receiptdate": "1992-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "mong the careful" }
+{ "l_orderkey": 5381, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47189.94, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-08", "l_commitdate": "1993-04-07", "l_receiptdate": "1993-06-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " accounts. regular, regula" }
 { "l_orderkey": 5575, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15408.96, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-10-14", "l_receiptdate": "1995-08-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "jole boldly beyond the final as" }
 { "l_orderkey": 5702, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29854.86, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-04", "l_commitdate": "1993-10-22", "l_receiptdate": "1994-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "pinto beans. blithely " }
-{ "l_orderkey": 354, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 34634.16, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "onic requests thrash bold g" }
-{ "l_orderkey": 513, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-07-31", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "efully ironic ideas doze slyl" }
-{ "l_orderkey": 1511, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30785.92, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. carefully ironi" }
-{ "l_orderkey": 3907, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21165.32, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly. furiously unusual deposits use afte" }
-{ "l_orderkey": 5027, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37520.34, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ess requests! quickly regular pac" }
-{ "l_orderkey": 1287, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9620.6, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ding, regular accounts" }
-{ "l_orderkey": 2087, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 962.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "hely final acc" }
-{ "l_orderkey": 4070, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10582.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-23", "l_commitdate": "1995-08-15", "l_receiptdate": "1995-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully final pack" }
-{ "l_orderkey": 4483, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48103.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ag blithely even" }
-{ "l_orderkey": 5382, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-26", "l_commitdate": "1992-02-17", "l_receiptdate": "1992-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully regular accounts. slyly ev" }
-{ "l_orderkey": 5888, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly final accounts hag" }
 { "l_orderkey": 33, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-19", "l_receiptdate": "1993-11-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ng to the furiously ironic package" }
-{ "l_orderkey": 482, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-01", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithe pin" }
-{ "l_orderkey": 1248, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28861.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fily special foxes kindle am" }
-{ "l_orderkey": 3936, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11544.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely across the carefully brave req" }
-{ "l_orderkey": 5378, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-17", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans sleep. fu" }
-{ "l_orderkey": 5536, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "equests mo" }
-{ "l_orderkey": 5602, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rate fluffily regular platelets. blithel" }
 { "l_orderkey": 71, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24051.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-10", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ckly. slyly" }
 { "l_orderkey": 102, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14430.9, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-02", "l_commitdate": "1997-07-13", "l_receiptdate": "1997-06-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "final packages. carefully even excu" }
+{ "l_orderkey": 354, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 34634.16, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-19", "l_commitdate": "1996-05-29", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "onic requests thrash bold g" }
+{ "l_orderkey": 482, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-01", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-06-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithe pin" }
+{ "l_orderkey": 513, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-07-31", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "efully ironic ideas doze slyl" }
+{ "l_orderkey": 1248, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28861.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-01", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "fily special foxes kindle am" }
+{ "l_orderkey": 1287, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9620.6, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-08-12", "l_receiptdate": "1994-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ding, regular accounts" }
+{ "l_orderkey": 1511, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30785.92, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-06", "l_commitdate": "1997-03-21", "l_receiptdate": "1997-01-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " deposits. carefully ironi" }
+{ "l_orderkey": 2087, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 962.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-11", "l_receiptdate": "1998-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "hely final acc" }
 { "l_orderkey": 3393, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16355.02, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-15", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "kly ironic deposits could" }
+{ "l_orderkey": 3907, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21165.32, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-20", "l_commitdate": "1992-10-30", "l_receiptdate": "1992-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly. furiously unusual deposits use afte" }
+{ "l_orderkey": 3936, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11544.72, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely across the carefully brave req" }
+{ "l_orderkey": 4070, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10582.66, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-23", "l_commitdate": "1995-08-15", "l_receiptdate": "1995-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " carefully final pack" }
 { "l_orderkey": 4450, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12506.78, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-09-18", "l_receiptdate": "1997-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " brave foxes. slyly unusual" }
 { "l_orderkey": 4453, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 46178.88, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-29", "l_commitdate": "1997-06-24", "l_receiptdate": "1997-06-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "eep. fluffily express accounts at the furi" }
+{ "l_orderkey": 4483, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 48103.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-07-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ag blithely even" }
+{ "l_orderkey": 5027, "l_partkey": 62, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37520.34, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-09", "l_commitdate": "1997-11-13", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ess requests! quickly regular pac" }
+{ "l_orderkey": 5378, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-17", "l_commitdate": "1993-01-20", "l_receiptdate": "1993-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans sleep. fu" }
+{ "l_orderkey": 5382, "l_partkey": 62, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-26", "l_commitdate": "1992-02-17", "l_receiptdate": "1992-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully regular accounts. slyly ev" }
+{ "l_orderkey": 5536, "l_partkey": 62, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 19241.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-08", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "equests mo" }
+{ "l_orderkey": 5602, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29823.86, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-10-24", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rate fluffily regular platelets. blithel" }
+{ "l_orderkey": 5888, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44254.76, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly final accounts hag" }
 { "l_orderkey": 5955, "l_partkey": 62, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14430.9, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-22", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-04-27", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y final accounts above the regu" }
-{ "l_orderkey": 291, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28831.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " fluffily regular deposits. quickl" }
-{ "l_orderkey": 899, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17299.08, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "re daring, pending deposits. blit" }
-{ "l_orderkey": 1380, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e ironic, even excuses haggle " }
-{ "l_orderkey": 2374, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1922.12, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", unusual ideas. deposits cajole quietl" }
-{ "l_orderkey": 2950, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 44208.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "to the regular accounts are slyly carefu" }
-{ "l_orderkey": 3682, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5766.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ronic deposits wake slyly. ca" }
-{ "l_orderkey": 3968, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6727.42, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-30", "l_commitdate": "1997-05-01", "l_receiptdate": "1997-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully bold instructions. express" }
-{ "l_orderkey": 5318, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12493.78, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-15", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly silent ideas. ideas haggle among the " }
-{ "l_orderkey": 5797, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16338.02, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ironic, even theodoli" }
 { "l_orderkey": 33, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30753.92, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-01-04", "l_receiptdate": "1993-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gular theodolites" }
-{ "l_orderkey": 262, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "atelets sleep furiously. requests cajole. b" }
-{ "l_orderkey": 1093, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32676.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-09-06", "l_receiptdate": "1997-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sits. express accounts play carefully. bol" }
-{ "l_orderkey": 1763, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 42286.64, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " instructions need to integrate deposits. " }
-{ "l_orderkey": 3463, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43247.7, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-04", "l_receiptdate": "1993-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "nts are slyly " }
-{ "l_orderkey": 5442, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11532.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-03-18", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully final" }
 { "l_orderkey": 261, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47091.94, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-29", "l_commitdate": "1993-09-08", "l_receiptdate": "1993-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " pinto beans haggle slyly furiously pending" }
-{ "l_orderkey": 1319, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20182.26, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s: carefully express " }
-{ "l_orderkey": 2117, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18260.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s between the slyly regula" }
-{ "l_orderkey": 2595, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40364.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ggle furiou" }
-{ "l_orderkey": 4167, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45169.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-08-24", "l_receiptdate": "1998-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " carefully final asymptotes. slyly bo" }
-{ "l_orderkey": 5312, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25948.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-04-09", "l_receiptdate": "1995-04-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tructions cajol" }
+{ "l_orderkey": 262, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-10", "l_commitdate": "1996-01-31", "l_receiptdate": "1996-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "atelets sleep furiously. requests cajole. b" }
+{ "l_orderkey": 291, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28831.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " fluffily regular deposits. quickl" }
 { "l_orderkey": 295, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24987.56, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-11-22", "l_receiptdate": "1995-01-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " carefully iron" }
+{ "l_orderkey": 899, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17299.08, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-06", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "re daring, pending deposits. blit" }
 { "l_orderkey": 999, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 32676.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-10-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "its. daringly final instruc" }
+{ "l_orderkey": 1093, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32676.04, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-07", "l_commitdate": "1997-09-06", "l_receiptdate": "1997-11-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sits. express accounts play carefully. bol" }
+{ "l_orderkey": 1319, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20182.26, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-12-02", "l_receiptdate": "1996-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s: carefully express " }
+{ "l_orderkey": 1380, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31714.98, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e ironic, even excuses haggle " }
+{ "l_orderkey": 1763, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 42286.64, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-06", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " instructions need to integrate deposits. " }
 { "l_orderkey": 2020, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25948.62, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-14", "l_commitdate": "1993-09-02", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e of the bold foxes haggle " }
+{ "l_orderkey": 2117, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18260.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-30", "l_commitdate": "1997-06-18", "l_receiptdate": "1997-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s between the slyly regula" }
 { "l_orderkey": 2370, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 30753.92, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-03-03", "l_receiptdate": "1994-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ies since the final deposits" }
+{ "l_orderkey": 2374, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1922.12, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", unusual ideas. deposits cajole quietl" }
+{ "l_orderkey": 2595, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40364.52, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ggle furiou" }
+{ "l_orderkey": 2950, "l_partkey": 61, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 44208.76, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "to the regular accounts are slyly carefu" }
+{ "l_orderkey": 3463, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43247.7, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-04", "l_receiptdate": "1993-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "nts are slyly " }
+{ "l_orderkey": 3682, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5766.36, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ronic deposits wake slyly. ca" }
 { "l_orderkey": 3846, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14415.9, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-17", "l_commitdate": "1998-04-27", "l_receiptdate": "1998-02-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uternes. carefully even" }
+{ "l_orderkey": 3968, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6727.42, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-30", "l_commitdate": "1997-05-01", "l_receiptdate": "1997-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "efully bold instructions. express" }
 { "l_orderkey": 3974, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16338.02, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-05", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ions eat slyly after the blithely " }
+{ "l_orderkey": 4167, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 45169.82, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-08-24", "l_receiptdate": "1998-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " carefully final asymptotes. slyly bo" }
 { "l_orderkey": 4672, "l_partkey": 61, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 39403.46, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-01", "l_commitdate": "1995-12-15", "l_receiptdate": "1995-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " slyly quie" }
+{ "l_orderkey": 5312, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25948.62, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-04-09", "l_receiptdate": "1995-04-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "tructions cajol" }
+{ "l_orderkey": 5318, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12493.78, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-15", "l_commitdate": "1993-06-25", "l_receiptdate": "1993-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly silent ideas. ideas haggle among the " }
 { "l_orderkey": 5376, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40364.52, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-20", "l_commitdate": "1994-08-30", "l_receiptdate": "1994-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y even asymptotes. courts are unusual pa" }
+{ "l_orderkey": 5442, "l_partkey": 61, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11532.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-03-18", "l_receiptdate": "1998-05-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fully final" }
+{ "l_orderkey": 5797, "l_partkey": 61, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16338.02, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-13", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ironic, even theodoli" }
 { "l_orderkey": 65, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24961.56, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-20", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-05-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pending deposits nag even packages. ca" }
-{ "l_orderkey": 2086, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21121.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-16", "l_receiptdate": "1994-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "idly busy acc" }
-{ "l_orderkey": 2849, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23041.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e slyly even asymptotes. slo" }
-{ "l_orderkey": 4743, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18241.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely even accounts" }
-{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4800.3, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ses integrate. regular deposits are about " }
-{ "l_orderkey": 4961, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 960.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s affix carefully silent dependen" }
-{ "l_orderkey": 5570, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 27841.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he silent, enticing requests." }
+{ "l_orderkey": 294, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29761.86, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-06", "l_commitdate": "1993-08-19", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "le fluffily along the quick" }
 { "l_orderkey": 579, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5760.36, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-24", "l_commitdate": "1998-05-03", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ickly final requests-- bold accou" }
-{ "l_orderkey": 1991, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47042.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-10", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests cajole blithely" }
-{ "l_orderkey": 2213, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3840.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " affix carefully furiously " }
-{ "l_orderkey": 2406, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 28801.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final pinto beans han" }
-{ "l_orderkey": 2817, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24001.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-21", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "doze blithely." }
-{ "l_orderkey": 3043, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40322.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ide of the un" }
-{ "l_orderkey": 4322, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 37442.34, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-16", "l_commitdate": "1998-05-21", "l_receiptdate": "1998-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts. dogged pin" }
 { "l_orderkey": 992, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13440.84, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1997-12-29", "l_receiptdate": "1998-02-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the unusual, even dependencies affix fluff" }
 { "l_orderkey": 1217, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 43202.7, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "riously close ideas" }
 { "l_orderkey": 1540, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33602.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-31", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e blithely a" }
+{ "l_orderkey": 1991, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 47042.94, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-10", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-10-07", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "quests cajole blithely" }
+{ "l_orderkey": 2086, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21121.32, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-12-16", "l_receiptdate": "1994-12-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "idly busy acc" }
 { "l_orderkey": 2149, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21121.32, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-24", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ptotes sleep along the blithely ir" }
+{ "l_orderkey": 2213, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3840.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " affix carefully furiously " }
 { "l_orderkey": 2305, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37442.34, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-16", "l_commitdate": "1993-04-17", "l_receiptdate": "1993-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ms after the foxes " }
-{ "l_orderkey": 2944, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16321.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly final dolphins sleep silent the" }
-{ "l_orderkey": 3013, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19201.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "unts boost regular ideas. slyly pe" }
-{ "l_orderkey": 3168, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44162.76, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-02", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the express accounts. fluff" }
-{ "l_orderkey": 4423, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1920.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-04", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old sheaves sleep" }
-{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17281.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "blithely ironic theodolites use along" }
-{ "l_orderkey": 294, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29761.86, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-06", "l_commitdate": "1993-08-19", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "le fluffily along the quick" }
+{ "l_orderkey": 2406, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 30.0, "l_extendedprice": 28801.8, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " final pinto beans han" }
+{ "l_orderkey": 2817, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 24001.5, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-21", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-05-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "doze blithely." }
+{ "l_orderkey": 2849, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23041.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-12", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e slyly even asymptotes. slo" }
 { "l_orderkey": 2886, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 960.06, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits fr" }
+{ "l_orderkey": 2944, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16321.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-18", "l_commitdate": "1997-11-27", "l_receiptdate": "1997-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly final dolphins sleep silent the" }
 { "l_orderkey": 3008, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 46082.88, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-23", "l_commitdate": "1996-01-07", "l_receiptdate": "1996-02-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld theodolites. fluffily bold theodolit" }
+{ "l_orderkey": 3013, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 19201.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "unts boost regular ideas. slyly pe" }
+{ "l_orderkey": 3043, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40322.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-15", "l_commitdate": "1992-06-19", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ide of the un" }
+{ "l_orderkey": 3168, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 44162.76, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-02", "l_receiptdate": "1992-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the express accounts. fluff" }
+{ "l_orderkey": 4322, "l_partkey": 60, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 37442.34, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-16", "l_commitdate": "1998-05-21", "l_receiptdate": "1998-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ccounts. dogged pin" }
+{ "l_orderkey": 4423, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1920.12, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-04", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old sheaves sleep" }
+{ "l_orderkey": 4743, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 18241.14, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-07-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely even accounts" }
+{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4800.3, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-07", "l_commitdate": "1994-07-15", "l_receiptdate": "1994-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ses integrate. regular deposits are about " }
+{ "l_orderkey": 4839, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17281.08, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "blithely ironic theodolites use along" }
+{ "l_orderkey": 4961, "l_partkey": 60, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 960.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-08", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s affix carefully silent dependen" }
+{ "l_orderkey": 5570, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 27841.74, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-20", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "he silent, enticing requests." }
 { "l_orderkey": 5735, "l_partkey": 60, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39362.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-23", "l_commitdate": "1995-02-10", "l_receiptdate": "1995-01-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lthily ruthless i" }
-{ "l_orderkey": 1571, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17262.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1993-01-12", "l_receiptdate": "1993-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " pending grouches " }
-{ "l_orderkey": 1605, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37402.95, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-08-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nal dependencies-- quickly final frets acc" }
-{ "l_orderkey": 1890, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "is wake carefully above the even id" }
-{ "l_orderkey": 2759, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9590.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. busily ironic theodo" }
-{ "l_orderkey": 2816, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-19", "l_commitdate": "1994-11-10", "l_receiptdate": "1994-11-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s; slyly even theodo" }
-{ "l_orderkey": 4672, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-03", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l instructions. blithely ironic packages " }
-{ "l_orderkey": 4998, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the blithely ironic " }
-{ "l_orderkey": 5472, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-07", "l_receiptdate": "1993-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily pending attainments. unus" }
 { "l_orderkey": 165, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " bold packages mainta" }
+{ "l_orderkey": 262, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 33566.75, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites cajole along the pending packag" }
 { "l_orderkey": 354, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47952.5, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-05-20", "l_receiptdate": "1996-04-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "to beans s" }
+{ "l_orderkey": 674, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3836.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-05", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly express pinto beans sleep car" }
 { "l_orderkey": 677, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30689.6, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-02-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly final" }
+{ "l_orderkey": 935, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " instructions. ironic acc" }
+{ "l_orderkey": 967, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39321.05, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ld foxes wake closely special" }
 { "l_orderkey": 1249, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46993.45, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-28", "l_receiptdate": "1994-03-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ffily express theodo" }
 { "l_orderkey": 1282, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18221.95, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-20", "l_commitdate": "1992-04-17", "l_receiptdate": "1992-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nto beans. carefully close theodo" }
-{ "l_orderkey": 1667, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-12-01", "l_receiptdate": "1997-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "hrash final requests. care" }
-{ "l_orderkey": 2087, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5754.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "dazzle after the slyly si" }
-{ "l_orderkey": 2688, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-09", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "press, ironic excuses wake carefully id" }
-{ "l_orderkey": 2791, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46993.45, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-11-10", "l_receiptdate": "1995-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts sleep at the bold, regular pinto " }
-{ "l_orderkey": 5157, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23976.25, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages detect. even requests against th" }
-{ "l_orderkey": 5444, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ut the courts cajole blithely excuses" }
-{ "l_orderkey": 935, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-14", "l_commitdate": "1997-11-22", "l_receiptdate": "1998-01-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " instructions. ironic acc" }
+{ "l_orderkey": 1284, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al packages use carefully express de" }
 { "l_orderkey": 1510, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-12-05", "l_receiptdate": "1996-11-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he blithely regular req" }
+{ "l_orderkey": 1571, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 17262.9, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1993-01-12", "l_receiptdate": "1993-01-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " pending grouches " }
+{ "l_orderkey": 1605, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37402.95, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-08-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nal dependencies-- quickly final frets acc" }
+{ "l_orderkey": 1667, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-14", "l_commitdate": "1997-12-01", "l_receiptdate": "1997-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "hrash final requests. care" }
+{ "l_orderkey": 1890, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 23017.2, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-09", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "is wake carefully above the even id" }
+{ "l_orderkey": 2087, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5754.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-03-27", "l_receiptdate": "1998-05-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "dazzle after the slyly si" }
 { "l_orderkey": 2406, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1997-01-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "hely even foxes unwind furiously aga" }
+{ "l_orderkey": 2688, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-09", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "press, ironic excuses wake carefully id" }
+{ "l_orderkey": 2759, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9590.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. busily ironic theodo" }
+{ "l_orderkey": 2791, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46993.45, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-11", "l_commitdate": "1994-11-10", "l_receiptdate": "1995-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " accounts sleep at the bold, regular pinto " }
+{ "l_orderkey": 2816, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-19", "l_commitdate": "1994-11-10", "l_receiptdate": "1994-11-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s; slyly even theodo" }
+{ "l_orderkey": 2945, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35484.85, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l instructions. regular, regular " }
+{ "l_orderkey": 3429, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans are fu" }
 { "l_orderkey": 3521, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 46034.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1992-12-31", "l_receiptdate": "1993-01-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ses use. furiously express ideas wake f" }
 { "l_orderkey": 3620, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39321.05, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-21", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-03-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "t attainments cajole qui" }
 { "l_orderkey": 3845, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16303.85, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-12", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "counts haggle. reg" }
-{ "l_orderkey": 5696, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ter the instruct" }
-{ "l_orderkey": 5957, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-23", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "platelets. furiously unusual requests " }
-{ "l_orderkey": 262, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 33566.75, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-02-14", "l_receiptdate": "1996-04-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lites cajole along the pending packag" }
-{ "l_orderkey": 674, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3836.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-05", "l_commitdate": "1992-11-22", "l_receiptdate": "1992-10-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly express pinto beans sleep car" }
-{ "l_orderkey": 967, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39321.05, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-08-15", "l_receiptdate": "1992-10-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ld foxes wake closely special" }
-{ "l_orderkey": 1284, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 959.05, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "al packages use carefully express de" }
-{ "l_orderkey": 2945, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35484.85, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-20", "l_receiptdate": "1996-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l instructions. regular, regular " }
-{ "l_orderkey": 3429, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-04", "l_commitdate": "1997-03-09", "l_receiptdate": "1997-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans are fu" }
 { "l_orderkey": 3906, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 34525.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-07", "l_commitdate": "1992-08-08", "l_receiptdate": "1992-08-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y. ironic deposits haggle sl" }
 { "l_orderkey": 4099, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 37402.95, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-13", "l_commitdate": "1992-11-13", "l_receiptdate": "1992-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fluffy accounts impress pending, iro" }
+{ "l_orderkey": 4672, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 21099.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-03", "l_commitdate": "1995-12-08", "l_receiptdate": "1995-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "l instructions. blithely ironic packages " }
+{ "l_orderkey": 4998, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-17", "l_commitdate": "1992-02-26", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the blithely ironic " }
+{ "l_orderkey": 5157, "l_partkey": 59, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23976.25, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-24", "l_commitdate": "1997-09-23", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages detect. even requests against th" }
+{ "l_orderkey": 5444, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31648.65, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-04-24", "l_receiptdate": "1995-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ut the courts cajole blithely excuses" }
+{ "l_orderkey": 5472, "l_partkey": 59, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25894.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-04", "l_commitdate": "1993-07-07", "l_receiptdate": "1993-09-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "fily pending attainments. unus" }
 { "l_orderkey": 5569, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14385.75, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "lithely bold requests boost fur" }
+{ "l_orderkey": 5696, "l_partkey": 59, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-10", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ter the instruct" }
 { "l_orderkey": 5923, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 33566.75, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-21", "l_commitdate": "1997-07-11", "l_receiptdate": "1997-08-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "sts affix unusual, final requests. request" }
+{ "l_orderkey": 5957, "l_partkey": 59, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 44116.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-23", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-02-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "platelets. furiously unusual requests " }
 { "l_orderkey": 935, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12454.65, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-13", "l_commitdate": "1997-11-30", "l_receiptdate": "1998-02-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ld platelet" }
-{ "l_orderkey": 2117, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41196.15, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " foxes sleep furiously " }
-{ "l_orderkey": 3111, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28741.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eas are furiously slyly special deposits." }
-{ "l_orderkey": 3586, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1916.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts. slyly final ideas agai" }
-{ "l_orderkey": 3685, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-16", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. special asymptotes about the r" }
-{ "l_orderkey": 4678, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33531.75, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-27", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he accounts. fluffily bold sheaves b" }
-{ "l_orderkey": 4869, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-07", "l_receiptdate": "1994-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites cajole after the ideas. special t" }
-{ "l_orderkey": 5920, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-21", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully regular dolphins. furiousl" }
-{ "l_orderkey": 2310, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34489.8, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-09", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "iously against the slyly special accounts" }
-{ "l_orderkey": 2501, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts. express, iron" }
-{ "l_orderkey": 3010, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar, even reques" }
-{ "l_orderkey": 3751, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11496.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "accounts wake furious" }
-{ "l_orderkey": 4997, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-04-23", "l_receiptdate": "1998-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xpress, bo" }
-{ "l_orderkey": 5024, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 39280.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits hinder carefully " }
-{ "l_orderkey": 5569, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-07-22", "l_receiptdate": "1993-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "pitaphs. ironic req" }
-{ "l_orderkey": 5575, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. slyly pending theodolites prin" }
-{ "l_orderkey": 5857, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23951.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-12-17", "l_receiptdate": "1997-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding platelets. pending excu" }
 { "l_orderkey": 1126, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-03-22", "l_receiptdate": "1998-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ons. final, unusual" }
 { "l_orderkey": 1412, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35447.85, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-04-19", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "hely express excuses are " }
-{ "l_orderkey": 2695, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15328.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "its. theodolites sleep slyly" }
-{ "l_orderkey": 4896, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5748.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-12", "l_receiptdate": "1992-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular deposits" }
-{ "l_orderkey": 5282, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26825.4, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "fily final instruc" }
-{ "l_orderkey": 5698, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14370.75, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly ironic frets haggle carefully " }
 { "l_orderkey": 1542, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35447.85, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-15", "l_commitdate": "1993-10-17", "l_receiptdate": "1994-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e blithely unusual accounts. quic" }
+{ "l_orderkey": 2117, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41196.15, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-27", "l_commitdate": "1997-06-12", "l_receiptdate": "1997-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " foxes sleep furiously " }
 { "l_orderkey": 2208, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45986.4, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-13", "l_commitdate": "1995-06-30", "l_receiptdate": "1995-05-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "sits. idly permanent request" }
+{ "l_orderkey": 2310, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34489.8, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-09", "l_commitdate": "1996-10-28", "l_receiptdate": "1996-10-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "iously against the slyly special accounts" }
+{ "l_orderkey": 2501, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-07-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts. express, iron" }
+{ "l_orderkey": 2695, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15328.8, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "its. theodolites sleep slyly" }
+{ "l_orderkey": 3010, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar, even reques" }
+{ "l_orderkey": 3111, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28741.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-11-15", "l_receiptdate": "1995-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eas are furiously slyly special deposits." }
 { "l_orderkey": 3360, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3832.2, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly busy inst" }
+{ "l_orderkey": 3586, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1916.1, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-04-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "unts. slyly final ideas agai" }
+{ "l_orderkey": 3685, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-16", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sits. special asymptotes about the r" }
+{ "l_orderkey": 3751, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11496.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "accounts wake furious" }
 { "l_orderkey": 4576, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 41196.15, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly final deposits. never" }
+{ "l_orderkey": 4678, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33531.75, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-27", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "he accounts. fluffily bold sheaves b" }
+{ "l_orderkey": 4869, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-17", "l_commitdate": "1994-11-07", "l_receiptdate": "1994-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "olites cajole after the ideas. special t" }
+{ "l_orderkey": 4896, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5748.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-11-12", "l_receiptdate": "1992-11-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular deposits" }
+{ "l_orderkey": 4997, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-20", "l_commitdate": "1998-04-23", "l_receiptdate": "1998-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "xpress, bo" }
+{ "l_orderkey": 5024, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 39280.05, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-12-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "osits hinder carefully " }
+{ "l_orderkey": 5282, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26825.4, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "fily final instruc" }
+{ "l_orderkey": 5569, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24909.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-07-22", "l_receiptdate": "1993-09-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "pitaphs. ironic req" }
+{ "l_orderkey": 5575, "l_partkey": 58, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6706.35, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-10-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. slyly pending theodolites prin" }
+{ "l_orderkey": 5698, "l_partkey": 58, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 14370.75, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-07-03", "l_receiptdate": "1994-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly ironic frets haggle carefully " }
 { "l_orderkey": 5796, "l_partkey": 58, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25867.35, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-02-29", "l_receiptdate": "1996-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s wake quickly aro" }
+{ "l_orderkey": 5857, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23951.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-12-17", "l_receiptdate": "1997-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding platelets. pending excu" }
+{ "l_orderkey": 5920, "l_partkey": 58, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22993.2, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1995-01-21", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "fully regular dolphins. furiousl" }
 { "l_orderkey": 194, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7656.4, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-06-25", "l_receiptdate": "1992-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "uriously unusual excuses" }
+{ "l_orderkey": 198, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully caref" }
+{ "l_orderkey": 231, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-05", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously special decoys wake q" }
 { "l_orderkey": 450, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 38282.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-04-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ve. asymptote" }
+{ "l_orderkey": 548, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " engage quickly. regular theo" }
+{ "l_orderkey": 582, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6699.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-29", "l_receiptdate": "1997-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ithely unusual t" }
 { "l_orderkey": 736, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12441.65, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-16", "l_commitdate": "1998-07-26", "l_receiptdate": "1998-08-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "st furiously among the " }
 { "l_orderkey": 805, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 27754.45, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-24", "l_commitdate": "1995-08-15", "l_receiptdate": "1995-09-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites according to the slyly f" }
-{ "l_orderkey": 2146, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40196.1, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-11-02", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns according to the doggedly " }
-{ "l_orderkey": 2630, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7656.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "indle fluffily silent, ironic pi" }
-{ "l_orderkey": 3072, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5742.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular requests abov" }
-{ "l_orderkey": 3682, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he requests cajole quickly pending package" }
-{ "l_orderkey": 3975, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36367.9, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es are furiously: furi" }
-{ "l_orderkey": 4068, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ds wake carefully amon" }
-{ "l_orderkey": 4672, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12441.65, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1995-12-13", "l_receiptdate": "1996-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests? pending accounts against" }
-{ "l_orderkey": 231, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-05", "l_commitdate": "1994-12-27", "l_receiptdate": "1994-11-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously special decoys wake q" }
-{ "l_orderkey": 582, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6699.35, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-16", "l_commitdate": "1997-11-29", "l_receiptdate": "1997-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ithely unusual t" }
 { "l_orderkey": 837, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37324.95, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-08-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ecial pinto bea" }
-{ "l_orderkey": 1444, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32539.7, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. doggedly pend" }
-{ "l_orderkey": 3270, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27754.45, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-22", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ptotes nag above the quickly bold deposits" }
-{ "l_orderkey": 3778, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. blithely special theodoli" }
-{ "l_orderkey": 4007, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30625.6, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-30", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nal accounts across t" }
-{ "l_orderkey": 5922, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37324.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e of the instructions. quick" }
-{ "l_orderkey": 1632, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-02-24", "l_receiptdate": "1997-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ructions! slyly" }
-{ "l_orderkey": 2215, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "against the carefu" }
-{ "l_orderkey": 198, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully caref" }
-{ "l_orderkey": 548, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-12-04", "l_receiptdate": "1994-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " engage quickly. regular theo" }
 { "l_orderkey": 962, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34453.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-09", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al foxes. iron" }
 { "l_orderkey": 962, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 19141.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-26", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits use fluffily according to " }
 { "l_orderkey": 964, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 46895.45, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-06", "l_commitdate": "1995-08-10", "l_receiptdate": "1995-10-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ounts. blithely regular packag" }
 { "l_orderkey": 1189, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 21055.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-09", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "quickly unusual platelets lose forges. ca" }
 { "l_orderkey": 1345, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-29", "l_receiptdate": "1992-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": ". slyly silent accounts sublat" }
+{ "l_orderkey": 1444, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32539.7, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-22", "l_commitdate": "1995-02-15", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y. doggedly pend" }
+{ "l_orderkey": 1632, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31582.65, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-01", "l_commitdate": "1997-02-24", "l_receiptdate": "1997-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ructions! slyly" }
 { "l_orderkey": 1664, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8613.45, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-15", "l_commitdate": "1996-05-14", "l_receiptdate": "1996-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ges. fluffil" }
 { "l_orderkey": 1924, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 38282.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-11-30", "l_receiptdate": "1996-11-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ains sleep carefully" }
+{ "l_orderkey": 2146, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 40196.1, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-11-02", "l_receiptdate": "1992-09-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ns according to the doggedly " }
 { "l_orderkey": 2177, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 44024.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-02-23", "l_receiptdate": "1997-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ending asymptotes." }
+{ "l_orderkey": 2215, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-09", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-09-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "against the carefu" }
 { "l_orderkey": 2372, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 18183.95, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " beans haggle sometimes" }
 { "l_orderkey": 2404, "l_partkey": 57, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18183.95, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-07", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "cuses. quickly even in" }
+{ "l_orderkey": 2630, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7656.4, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "indle fluffily silent, ironic pi" }
+{ "l_orderkey": 3072, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5742.3, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-03-24", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "gular requests abov" }
 { "l_orderkey": 3110, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-31", "l_commitdate": "1995-03-07", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "en deposits. ironic" }
+{ "l_orderkey": 3270, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27754.45, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-22", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ptotes nag above the quickly bold deposits" }
 { "l_orderkey": 3271, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r the unusual Tiresia" }
+{ "l_orderkey": 3682, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28711.5, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he requests cajole quickly pending package" }
+{ "l_orderkey": 3778, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20098.05, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-10", "l_receiptdate": "1993-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ts. blithely special theodoli" }
+{ "l_orderkey": 3975, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36367.9, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-02", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "es are furiously: furi" }
+{ "l_orderkey": 4007, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30625.6, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-30", "l_commitdate": "1993-08-16", "l_receiptdate": "1993-10-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nal accounts across t" }
 { "l_orderkey": 4034, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44981.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1993-12-26", "l_receiptdate": "1994-02-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "eodolites was slyly ironic ideas. de" }
+{ "l_orderkey": 4068, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29668.55, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ds wake carefully amon" }
 { "l_orderkey": 4096, "l_partkey": 57, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16269.85, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-08-11", "l_receiptdate": "1992-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets alongside of the " }
-{ "l_orderkey": 903, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8604.45, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he slyly ev" }
-{ "l_orderkey": 1638, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-27", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " quickly expres" }
-{ "l_orderkey": 708, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c pinto beans nag after the account" }
-{ "l_orderkey": 1248, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24857.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " ironic dependen" }
-{ "l_orderkey": 2272, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34417.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ironic packages; quickly iron" }
-{ "l_orderkey": 2752, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3824.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-01-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "telets haggle. regular, final " }
-{ "l_orderkey": 3685, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35373.85, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-03-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully sly requests are regular, regu" }
-{ "l_orderkey": 5190, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41110.15, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies use fluffily unusual requests? hoc" }
-{ "l_orderkey": 5411, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-12", "l_commitdate": "1997-08-03", "l_receiptdate": "1997-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " bold, ironic theodo" }
-{ "l_orderkey": 5697, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40154.1, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-12-08", "l_receiptdate": "1993-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "inal theodolites cajole after the bli" }
-{ "l_orderkey": 387, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-21", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular dependencies" }
-{ "l_orderkey": 2531, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26769.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-07-31", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its. busily" }
-{ "l_orderkey": 4421, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43978.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "reful packages. bold, " }
-{ "l_orderkey": 4705, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15296.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special ideas nag sl" }
-{ "l_orderkey": 4996, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33461.75, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. unusual, regular dolphins integrate care" }
+{ "l_orderkey": 4672, "l_partkey": 57, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 12441.65, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-02", "l_commitdate": "1995-12-13", "l_receiptdate": "1996-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar requests? pending accounts against" }
+{ "l_orderkey": 5922, "l_partkey": 57, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 37324.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1997-01-17", "l_receiptdate": "1997-03-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e of the instructions. quick" }
 { "l_orderkey": 70, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-02-17", "l_receiptdate": "1994-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages wake pending accounts." }
 { "l_orderkey": 356, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 39198.05, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-28", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " according to the express foxes will" }
+{ "l_orderkey": 387, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-04-21", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular dependencies" }
+{ "l_orderkey": 708, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-22", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-07-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "c pinto beans nag after the account" }
+{ "l_orderkey": 903, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8604.45, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-06", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-10-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he slyly ev" }
+{ "l_orderkey": 1248, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24857.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-01", "l_receiptdate": "1992-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " ironic dependen" }
+{ "l_orderkey": 1638, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18164.95, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-10-27", "l_receiptdate": "1997-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " quickly expres" }
+{ "l_orderkey": 2272, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34417.8, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-25", "l_commitdate": "1993-05-23", "l_receiptdate": "1993-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "about the ironic packages; quickly iron" }
 { "l_orderkey": 2308, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34417.8, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-11", "l_commitdate": "1992-11-27", "l_receiptdate": "1992-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ong the pending hockey players. blithe" }
+{ "l_orderkey": 2531, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26769.4, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-07-31", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its. busily" }
+{ "l_orderkey": 2752, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3824.2, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-01-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "telets haggle. regular, final " }
 { "l_orderkey": 3205, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9560.5, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-04", "l_receiptdate": "1992-07-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " deposits cajole careful" }
 { "l_orderkey": 3590, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24857.3, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-08", "l_commitdate": "1995-06-17", "l_receiptdate": "1995-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully along th" }
 { "l_orderkey": 3680, "l_partkey": 56, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31549.65, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-16", "l_commitdate": "1993-02-19", "l_receiptdate": "1993-04-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ts. ironic, fina" }
+{ "l_orderkey": 3685, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35373.85, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-04-10", "l_receiptdate": "1992-03-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". carefully sly requests are regular, regu" }
 { "l_orderkey": 4129, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30593.6, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-08-25", "l_receiptdate": "1993-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ckages haggl" }
+{ "l_orderkey": 4421, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43978.3, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "reful packages. bold, " }
 { "l_orderkey": 4450, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5736.3, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-09-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "eposits. foxes cajole unusual fox" }
+{ "l_orderkey": 4705, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15296.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-02", "l_commitdate": "1992-06-06", "l_receiptdate": "1992-07-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "special ideas nag sl" }
+{ "l_orderkey": 4996, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33461.75, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-30", "l_commitdate": "1992-10-27", "l_receiptdate": "1992-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. unusual, regular dolphins integrate care" }
+{ "l_orderkey": 5190, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41110.15, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-19", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "encies use fluffily unusual requests? hoc" }
 { "l_orderkey": 5347, "l_partkey": 56, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5736.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-11", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-05-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lly unusual ideas. sl" }
-{ "l_orderkey": 928, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 47752.5, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " slyly slyly special request" }
-{ "l_orderkey": 4519, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28651.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-11", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "totes. slyly bold somas after the " }
-{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully ironic dep" }
-{ "l_orderkey": 5124, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41067.15, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-10", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "onic package" }
-{ "l_orderkey": 5186, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y ruthless foxes. fluffily " }
-{ "l_orderkey": 5956, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21966.15, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly slyly special " }
-{ "l_orderkey": 871, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44887.35, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-01", "l_receiptdate": "1996-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss, final dep" }
-{ "l_orderkey": 2022, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions dazzle carefull" }
-{ "l_orderkey": 2177, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32471.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-03", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tes are doggedly quickly" }
-{ "l_orderkey": 2180, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nic instructions haggle careful" }
-{ "l_orderkey": 5157, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33426.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously sil" }
+{ "l_orderkey": 5411, "l_partkey": 56, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4780.25, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-12", "l_commitdate": "1997-08-03", "l_receiptdate": "1997-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " bold, ironic theodo" }
+{ "l_orderkey": 5697, "l_partkey": 56, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40154.1, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-12-08", "l_receiptdate": "1993-01-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "inal theodolites cajole after the bli" }
 { "l_orderkey": 39, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 41067.15, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-17", "l_commitdate": "1996-11-14", "l_receiptdate": "1996-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "yly regular i" }
-{ "l_orderkey": 2181, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26741.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-23", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s excuses sleep car" }
-{ "l_orderkey": 2849, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-06-05", "l_receiptdate": "1996-05-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "mong the carefully regular theodol" }
-{ "l_orderkey": 3399, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7640.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s use carefully carefully ir" }
-{ "l_orderkey": 4006, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10505.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ress foxes cajole quick" }
-{ "l_orderkey": 4672, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42977.25, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " platelets use amon" }
-{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40112.1, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-28", "l_receiptdate": "1992-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sts. blithely regular deposits will have" }
-{ "l_orderkey": 5382, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12415.65, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-12", "l_receiptdate": "1992-02-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eodolites. final foxes " }
-{ "l_orderkey": 5697, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22921.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-27", "l_commitdate": "1992-11-28", "l_receiptdate": "1992-11-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uffily iro" }
-{ "l_orderkey": 5699, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake fluffily u" }
+{ "l_orderkey": 871, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44887.35, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-01", "l_receiptdate": "1996-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ss, final dep" }
+{ "l_orderkey": 928, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 47752.5, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " slyly slyly special request" }
 { "l_orderkey": 964, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 42022.2, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-10-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ronic deposit" }
 { "l_orderkey": 1408, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-19", "l_commitdate": "1998-03-14", "l_receiptdate": "1998-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ic foxes ca" }
 { "l_orderkey": 1574, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23876.25, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-16", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-02-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly silent accounts." }
 { "l_orderkey": 1856, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9550.5, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-11", "l_commitdate": "1992-05-20", "l_receiptdate": "1992-06-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "he furiously even theodolites. account" }
+{ "l_orderkey": 2022, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-17", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions dazzle carefull" }
+{ "l_orderkey": 2177, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32471.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-03", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tes are doggedly quickly" }
+{ "l_orderkey": 2180, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-11-22", "l_receiptdate": "1997-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nic instructions haggle careful" }
+{ "l_orderkey": 2181, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26741.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-21", "l_commitdate": "1995-10-23", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s excuses sleep car" }
 { "l_orderkey": 2657, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10505.55, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-19", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly enticing requests. fur" }
+{ "l_orderkey": 2849, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-06-05", "l_receiptdate": "1996-05-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "mong the carefully regular theodol" }
+{ "l_orderkey": 3399, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7640.4, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-15", "l_commitdate": "1995-04-19", "l_receiptdate": "1995-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s use carefully carefully ir" }
 { "l_orderkey": 3588, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26741.4, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-23", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-04-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " express sheaves. unusual theodo" }
+{ "l_orderkey": 4006, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10505.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-05-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ress foxes cajole quick" }
+{ "l_orderkey": 4519, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28651.5, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-11", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-04-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "totes. slyly bold somas after the " }
+{ "l_orderkey": 4672, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42977.25, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-07", "l_commitdate": "1996-01-16", "l_receiptdate": "1996-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " platelets use amon" }
+{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-12-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": ". carefully ironic dep" }
+{ "l_orderkey": 4897, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 40112.1, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-10-28", "l_receiptdate": "1992-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "sts. blithely regular deposits will have" }
+{ "l_orderkey": 5124, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 41067.15, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-10", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "onic package" }
+{ "l_orderkey": 5157, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33426.75, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-08-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "to the furiously sil" }
+{ "l_orderkey": 5186, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 36291.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "y ruthless foxes. fluffily " }
+{ "l_orderkey": 5382, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12415.65, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-01-16", "l_commitdate": "1992-03-12", "l_receiptdate": "1992-02-06", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "eodolites. final foxes " }
 { "l_orderkey": 5569, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45842.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-06-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the fluffily" }
 { "l_orderkey": 5605, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 42977.25, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-09-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly unusual instructions. carefully ironic p" }
+{ "l_orderkey": 5697, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22921.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-27", "l_commitdate": "1992-11-28", "l_receiptdate": "1992-11-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uffily iro" }
+{ "l_orderkey": 5699, "l_partkey": 55, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24831.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-11", "l_commitdate": "1992-09-21", "l_receiptdate": "1992-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final deposits wake fluffily u" }
 { "l_orderkey": 5699, "l_partkey": 55, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 43932.3, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-28", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-12-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "o the slyly" }
+{ "l_orderkey": 5956, "l_partkey": 55, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21966.15, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly slyly special " }
+{ "l_orderkey": 100, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35299.85, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nd the quickly s" }
 { "l_orderkey": 385, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43886.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-29", "l_commitdate": "1996-05-17", "l_receiptdate": "1996-04-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "lthily ironic f" }
+{ "l_orderkey": 642, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24805.3, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests according to the unu" }
+{ "l_orderkey": 770, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-23", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits dazzle fluffily alongside of " }
+{ "l_orderkey": 772, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 40070.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-16", "l_receiptdate": "1993-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express foxes abo" }
+{ "l_orderkey": 803, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7632.4, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-08-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ronic theodo" }
 { "l_orderkey": 833, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 954.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-04-05", "l_receiptdate": "1994-04-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ffily ironic theodolites" }
 { "l_orderkey": 1253, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12402.65, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1993-04-26", "l_receiptdate": "1993-03-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "al packages" }
 { "l_orderkey": 1346, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12402.65, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-22", "l_commitdate": "1992-08-10", "l_receiptdate": "1992-08-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "arefully brave deposits into the slyly iro" }
 { "l_orderkey": 1350, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 20035.05, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-17", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-12-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lyly above the evenly " }
-{ "l_orderkey": 2535, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4770.25, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " across the express requests. silent, eve" }
-{ "l_orderkey": 3427, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39116.05, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s the carefully" }
-{ "l_orderkey": 3968, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25759.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t silently." }
-{ "l_orderkey": 5408, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45794.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular " }
-{ "l_orderkey": 5701, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16218.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-04-08", "l_receiptdate": "1997-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tes. quickly final a" }
-{ "l_orderkey": 642, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 24805.3, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "quests according to the unu" }
-{ "l_orderkey": 803, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7632.4, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-04", "l_commitdate": "1997-06-19", "l_receiptdate": "1997-08-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ronic theodo" }
-{ "l_orderkey": 3046, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43886.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits sleep furious" }
-{ "l_orderkey": 3111, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13356.7, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "re. pinto " }
-{ "l_orderkey": 5925, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28621.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " the packa" }
-{ "l_orderkey": 100, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35299.85, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-04-16", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "nd the quickly s" }
-{ "l_orderkey": 772, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 40070.1, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-07-16", "l_receiptdate": "1993-06-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " express foxes abo" }
-{ "l_orderkey": 3271, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17172.9, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-28", "l_receiptdate": "1992-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages eat around the furiously regul" }
-{ "l_orderkey": 3749, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9540.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "essly. regular pi" }
-{ "l_orderkey": 4034, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41024.15, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1994-01-08", "l_receiptdate": "1993-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits wake carefully af" }
-{ "l_orderkey": 4321, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 42932.25, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " haggle ironically bold theodolites. quick" }
-{ "l_orderkey": 4645, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-12-11", "l_receiptdate": "1994-11-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "braids. ironic dependencies main" }
-{ "l_orderkey": 4865, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31483.65, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y pending notornis ab" }
-{ "l_orderkey": 770, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-07-23", "l_receiptdate": "1998-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " deposits dazzle fluffily alongside of " }
 { "l_orderkey": 1473, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47702.5, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-05", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-05-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "requests wake express deposits. special, ir" }
 { "l_orderkey": 1701, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1908.1, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-24", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ween the pending, final accounts. " }
 { "l_orderkey": 1988, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7632.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-20", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "le quickly ac" }
 { "l_orderkey": 2150, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37207.95, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-08-17", "l_receiptdate": "1994-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ess accounts nag. unusual asymptotes haggl" }
 { "l_orderkey": 2311, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14310.75, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-06", "l_receiptdate": "1995-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ve the blithely pending accounts. furio" }
 { "l_orderkey": 2533, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34345.8, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-10", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-07-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ss requests sleep neve" }
+{ "l_orderkey": 2535, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4770.25, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-08-14", "l_receiptdate": "1993-08-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " across the express requests. silent, eve" }
 { "l_orderkey": 2784, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21943.15, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests lose after " }
+{ "l_orderkey": 3046, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43886.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-22", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "sits sleep furious" }
+{ "l_orderkey": 3111, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13356.7, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-17", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "re. pinto " }
+{ "l_orderkey": 3271, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17172.9, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-01", "l_commitdate": "1992-03-28", "l_receiptdate": "1992-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " packages eat around the furiously regul" }
+{ "l_orderkey": 3427, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 39116.05, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-11", "l_commitdate": "1997-07-03", "l_receiptdate": "1997-10-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s the carefully" }
+{ "l_orderkey": 3749, "l_partkey": 54, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9540.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-24", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-07-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "essly. regular pi" }
+{ "l_orderkey": 3968, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25759.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-25", "l_commitdate": "1997-04-17", "l_receiptdate": "1997-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t silently." }
+{ "l_orderkey": 4034, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 41024.15, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1994-01-08", "l_receiptdate": "1993-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "posits wake carefully af" }
+{ "l_orderkey": 4321, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 42932.25, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-13", "l_commitdate": "1994-09-15", "l_receiptdate": "1994-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " haggle ironically bold theodolites. quick" }
 { "l_orderkey": 4515, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 30529.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-07", "l_commitdate": "1992-05-11", "l_receiptdate": "1992-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully express depo" }
+{ "l_orderkey": 4645, "l_partkey": 54, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23851.25, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-25", "l_commitdate": "1994-12-11", "l_receiptdate": "1994-11-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "braids. ironic dependencies main" }
+{ "l_orderkey": 4865, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31483.65, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-07-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y pending notornis ab" }
 { "l_orderkey": 5350, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11448.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-30", "l_commitdate": "1993-11-21", "l_receiptdate": "1994-02-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " cajole. even instructions haggle. blithe" }
+{ "l_orderkey": 5408, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45794.4, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-30", "l_commitdate": "1992-08-27", "l_receiptdate": "1992-10-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": ". furiously regular " }
 { "l_orderkey": 5412, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1908.1, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-04-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " sleep above the furiou" }
+{ "l_orderkey": 5701, "l_partkey": 54, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16218.85, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-27", "l_commitdate": "1997-04-08", "l_receiptdate": "1997-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tes. quickly final a" }
+{ "l_orderkey": 5925, "l_partkey": 54, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 28621.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " the packa" }
 { "l_orderkey": 480, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20967.1, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-07-28", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "into beans cajole furiously. accounts s" }
+{ "l_orderkey": 772, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33356.75, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "kly thin packages wake slowly" }
+{ "l_orderkey": 1060, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 953.05, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits detect carefully abo" }
 { "l_orderkey": 1698, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35262.85, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-16", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-05-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ly regular ideas. deposit" }
+{ "l_orderkey": 1731, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35262.85, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-30", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " beans use furiously slyly b" }
+{ "l_orderkey": 1888, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45746.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ar ideas cajole. regular p" }
+{ "l_orderkey": 1893, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5718.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ar accounts use. daringly ironic packag" }
+{ "l_orderkey": 1952, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6671.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-05-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "about the express, even requ" }
 { "l_orderkey": 1984, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42887.25, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "p. quickly final ideas sle" }
 { "l_orderkey": 1991, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46699.45, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-11-29", "l_receiptdate": "1992-10-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nd the ideas affi" }
 { "l_orderkey": 2240, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 37168.95, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-22", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-06-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y orbits. final depos" }
-{ "l_orderkey": 2950, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13342.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ccounts haggle carefully according " }
-{ "l_orderkey": 4036, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20014.05, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-28", "l_receiptdate": "1997-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e carefully. qui" }
-{ "l_orderkey": 1893, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 5718.3, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-02-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ar accounts use. daringly ironic packag" }
-{ "l_orderkey": 1952, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6671.35, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-06", "l_commitdate": "1994-06-11", "l_receiptdate": "1994-05-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "about the express, even requ" }
+{ "l_orderkey": 2246, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20967.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-08-03", "l_receiptdate": "1996-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ructions wake carefully fina" }
 { "l_orderkey": 2400, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21920.15, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-08-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tions. fluffily ironic platelets cajole c" }
+{ "l_orderkey": 2532, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "unusual sentiments. even pinto" }
 { "l_orderkey": 2562, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26685.4, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-04", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ans haggle special, special packages. " }
+{ "l_orderkey": 2950, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13342.7, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-05", "l_receiptdate": "1997-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ccounts haggle carefully according " }
+{ "l_orderkey": 3717, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nside the regular packages sleep" }
+{ "l_orderkey": 4036, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 20014.05, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-06-28", "l_receiptdate": "1997-08-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e carefully. qui" }
+{ "l_orderkey": 4800, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22873.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-14", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully carefully r" }
+{ "l_orderkey": 4967, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40981.15, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ons. slyly ironic requests" }
 { "l_orderkey": 5153, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13342.7, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-12-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " slyly daring pinto beans lose blithely fi" }
 { "l_orderkey": 5664, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29544.55, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-10", "l_commitdate": "1998-09-12", "l_receiptdate": "1998-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ainst the never silent request" }
 { "l_orderkey": 5793, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19061.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-09-04", "l_receiptdate": "1997-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e carefully ex" }
 { "l_orderkey": 5924, "l_partkey": 53, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46699.45, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-25", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-11-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "inly final excuses. blithely regular requ" }
-{ "l_orderkey": 772, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33356.75, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-05", "l_receiptdate": "1993-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "kly thin packages wake slowly" }
-{ "l_orderkey": 1731, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 35262.85, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-30", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " beans use furiously slyly b" }
-{ "l_orderkey": 1888, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45746.4, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-28", "l_commitdate": "1993-12-16", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ar ideas cajole. regular p" }
-{ "l_orderkey": 2246, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20967.1, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-25", "l_commitdate": "1996-08-03", "l_receiptdate": "1996-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ructions wake carefully fina" }
-{ "l_orderkey": 4800, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22873.2, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-01-14", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully carefully r" }
-{ "l_orderkey": 1060, "l_partkey": 53, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 953.05, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-19", "l_commitdate": "1993-05-10", "l_receiptdate": "1993-06-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits detect carefully abo" }
-{ "l_orderkey": 2532, "l_partkey": 53, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "unusual sentiments. even pinto" }
-{ "l_orderkey": 3717, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2859.15, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-09", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-06-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nside the regular packages sleep" }
-{ "l_orderkey": 4967, "l_partkey": 53, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40981.15, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-28", "l_commitdate": "1997-04-10", "l_receiptdate": "1997-06-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ons. slyly ironic requests" }
+{ "l_orderkey": 928, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 40938.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-14", "l_commitdate": "1995-04-21", "l_receiptdate": "1995-05-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely. express, silent requests doze at" }
+{ "l_orderkey": 1057, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18088.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r-- packages haggle alon" }
+{ "l_orderkey": 1280, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22849.2, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y pending orbits boost after the slyly" }
 { "l_orderkey": 1313, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45698.4, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-20", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "s are quick" }
 { "l_orderkey": 1571, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1993-02-24", "l_receiptdate": "1993-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ng to the fluffily unusual " }
-{ "l_orderkey": 2279, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " above the furiously ironic deposits. " }
-{ "l_orderkey": 3075, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1904.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". unusual, unusual accounts haggle furious" }
-{ "l_orderkey": 3907, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42842.25, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " about the regular pac" }
-{ "l_orderkey": 4388, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12376.65, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly even, expre" }
-{ "l_orderkey": 928, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 40938.15, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-14", "l_commitdate": "1995-04-21", "l_receiptdate": "1995-05-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely. express, silent requests doze at" }
-{ "l_orderkey": 2019, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "are carefully furiously regular requ" }
-{ "l_orderkey": 2183, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23801.25, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-08-21", "l_receiptdate": "1996-08-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he quickly f" }
-{ "l_orderkey": 3111, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9520.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng the slyly ironic inst" }
-{ "l_orderkey": 3969, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37129.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-06-13", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly bold ideas s" }
-{ "l_orderkey": 4003, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-02", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ar grouches s" }
-{ "l_orderkey": 4611, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously. furiously regular" }
-{ "l_orderkey": 4934, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven, ironic ideas" }
 { "l_orderkey": 1637, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19993.05, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-30", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly ironic theodolites use b" }
+{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31417.65, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-01-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. excuses a" }
+{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35225.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-17", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " integrate. quickly unusual" }
 { "l_orderkey": 1767, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 38082.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-05-06", "l_receiptdate": "1995-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ep. accounts nag blithely fu" }
 { "l_orderkey": 1799, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7616.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ealms upon the special, ironic waters" }
+{ "l_orderkey": 2019, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-24", "l_commitdate": "1992-12-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "are carefully furiously regular requ" }
+{ "l_orderkey": 2183, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23801.25, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-08-21", "l_receiptdate": "1996-08-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he quickly f" }
+{ "l_orderkey": 2279, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " above the furiously ironic deposits. " }
 { "l_orderkey": 2338, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28561.5, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-10", "l_commitdate": "1997-10-15", "l_receiptdate": "1997-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ould have to nag quickly" }
 { "l_orderkey": 2567, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5712.3, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-21", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-05-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s cajole regular, final acco" }
 { "l_orderkey": 2945, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "quests use" }
 { "l_orderkey": 3040, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40938.15, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-25", "l_receiptdate": "1993-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "sts nag slyly alongside of the depos" }
-{ "l_orderkey": 4454, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45698.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-23", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans wake across th" }
-{ "l_orderkey": 4838, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24753.3, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular requests boost about the packages. r" }
-{ "l_orderkey": 5159, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4760.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal deposits. pending, ironic ideas grow" }
-{ "l_orderkey": 1057, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 19.0, "l_extendedprice": 18088.95, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-31", "l_commitdate": "1992-05-09", "l_receiptdate": "1992-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "r-- packages haggle alon" }
-{ "l_orderkey": 1280, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22849.2, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-20", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y pending orbits boost after the slyly" }
-{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 31417.65, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-03", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-01-31", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s. excuses a" }
-{ "l_orderkey": 1761, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35225.85, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-17", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-03-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " integrate. quickly unusual" }
+{ "l_orderkey": 3075, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1904.1, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-06-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". unusual, unusual accounts haggle furious" }
 { "l_orderkey": 3106, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nstructions wake. furiously " }
+{ "l_orderkey": 3111, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9520.5, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-10", "l_commitdate": "1995-11-02", "l_receiptdate": "1995-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ng the slyly ironic inst" }
 { "l_orderkey": 3430, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 21897.15, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-03-01", "l_receiptdate": "1995-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "eas according to the" }
+{ "l_orderkey": 3907, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42842.25, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-21", "l_commitdate": "1992-09-19", "l_receiptdate": "1992-10-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " about the regular pac" }
+{ "l_orderkey": 3969, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 37129.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-06-13", "l_receiptdate": "1997-07-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ly bold ideas s" }
+{ "l_orderkey": 4003, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 17136.9, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-02", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ar grouches s" }
+{ "l_orderkey": 4388, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12376.65, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-28", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly even, expre" }
 { "l_orderkey": 4448, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22849.2, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-09", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal packages along the ironic instructi" }
+{ "l_orderkey": 4454, "l_partkey": 52, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45698.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-23", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "to beans wake across th" }
+{ "l_orderkey": 4611, "l_partkey": 52, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44746.35, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-05", "l_commitdate": "1993-03-01", "l_receiptdate": "1993-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously. furiously regular" }
+{ "l_orderkey": 4838, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24753.3, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-03", "l_commitdate": "1992-10-25", "l_receiptdate": "1992-09-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular requests boost about the packages. r" }
+{ "l_orderkey": 4934, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39986.1, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ven, ironic ideas" }
+{ "l_orderkey": 5159, "l_partkey": 52, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4760.25, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal deposits. pending, ironic ideas grow" }
 { "l_orderkey": 5282, "l_partkey": 52, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30465.6, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-01", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-03-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "onic deposits; furiou" }
 { "l_orderkey": 224, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3804.2, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-08", "l_commitdate": "1994-08-24", "l_receiptdate": "1994-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions " }
-{ "l_orderkey": 1347, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pin" }
-{ "l_orderkey": 2144, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43748.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes haggle blithel" }
-{ "l_orderkey": 2561, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13314.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-07", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep unusual, ironic accounts" }
-{ "l_orderkey": 2690, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-13", "l_commitdate": "1996-05-22", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " doubt careful" }
-{ "l_orderkey": 3233, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21874.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1995-01-11", "l_receiptdate": "1994-12-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pending instructions use after the carefu" }
-{ "l_orderkey": 4193, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27580.45, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. final packages use blit" }
-{ "l_orderkey": 4354, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1994-12-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s nag quickly " }
-{ "l_orderkey": 4544, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-09-07", "l_receiptdate": "1997-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular packages. s" }
-{ "l_orderkey": 4836, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11412.6, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sly ironic accoun" }
-{ "l_orderkey": 4932, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12363.65, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-10-16", "l_receiptdate": "1993-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "slyly according to the furiously fin" }
+{ "l_orderkey": 288, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "instructions wa" }
 { "l_orderkey": 512, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-19", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e slyly silent accounts serve with" }
+{ "l_orderkey": 582, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46601.45, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nts according to the furiously regular pin" }
 { "l_orderkey": 643, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45650.4, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-10", "l_commitdate": "1995-06-07", "l_receiptdate": "1995-08-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly ironic accounts" }
+{ "l_orderkey": 644, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36139.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages. blithely slow accounts nag quic" }
 { "l_orderkey": 676, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8559.45, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-03", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-04-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "aintain sl" }
 { "l_orderkey": 1189, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21874.15, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-08-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. fluffy Tiresias run quickly. bra" }
+{ "l_orderkey": 1347, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-04", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-07-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "y ironic pin" }
+{ "l_orderkey": 1508, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15216.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-05-30", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously across the ironic, unusua" }
+{ "l_orderkey": 1731, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly slyly speci" }
+{ "l_orderkey": 1920, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lly. ideas wa" }
+{ "l_orderkey": 1926, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22825.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-04", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e theodolites." }
+{ "l_orderkey": 2115, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2853.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quickly ironic dolphin" }
+{ "l_orderkey": 2144, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43748.3, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-04-29", "l_receiptdate": "1994-05-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " foxes haggle blithel" }
 { "l_orderkey": 2244, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2853.15, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-30", "l_commitdate": "1993-03-15", "l_receiptdate": "1993-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " beans for the regular platel" }
 { "l_orderkey": 2305, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6657.35, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-15", "l_commitdate": "1993-04-25", "l_receiptdate": "1993-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "gular deposits boost about the foxe" }
+{ "l_orderkey": 2465, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32335.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. regular package" }
+{ "l_orderkey": 2561, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 13314.7, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-07", "l_commitdate": "1998-02-04", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep unusual, ironic accounts" }
+{ "l_orderkey": 2690, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-13", "l_commitdate": "1996-05-22", "l_receiptdate": "1996-06-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " doubt careful" }
+{ "l_orderkey": 2786, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39944.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-15", "l_commitdate": "1992-04-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "unts are against the furious" }
 { "l_orderkey": 2951, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14265.75, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-25", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "inal account" }
 { "l_orderkey": 3104, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 19021.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-24", "l_receiptdate": "1994-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s are. furiously s" }
-{ "l_orderkey": 4324, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41846.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ainst the u" }
-{ "l_orderkey": 5860, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ual patterns try to eat carefully above" }
-{ "l_orderkey": 644, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 36139.9, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-17", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages. blithely slow accounts nag quic" }
-{ "l_orderkey": 1731, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47552.5, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly slyly speci" }
-{ "l_orderkey": 1926, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22825.2, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-04", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e theodolites." }
-{ "l_orderkey": 4741, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "t, regular requests" }
-{ "l_orderkey": 5088, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38993.05, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-07", "l_receiptdate": "1993-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ing requests. " }
-{ "l_orderkey": 288, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-04-28", "l_receiptdate": "1997-04-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "instructions wa" }
-{ "l_orderkey": 582, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 46601.45, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nts according to the furiously regular pin" }
-{ "l_orderkey": 1508, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15216.8, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-05-30", "l_receiptdate": "1998-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously across the ironic, unusua" }
-{ "l_orderkey": 1920, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 29482.55, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-30", "l_receiptdate": "1998-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "lly. ideas wa" }
-{ "l_orderkey": 2115, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2853.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-23", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "quickly ironic dolphin" }
-{ "l_orderkey": 2465, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32335.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-02", "l_commitdate": "1995-08-04", "l_receiptdate": "1995-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. regular package" }
-{ "l_orderkey": 2786, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39944.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-15", "l_commitdate": "1992-04-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "unts are against the furious" }
+{ "l_orderkey": 3233, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21874.15, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1995-01-11", "l_receiptdate": "1994-12-26", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pending instructions use after the carefu" }
 { "l_orderkey": 3972, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-24", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y final theodolite" }
 { "l_orderkey": 4099, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34237.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-06", "l_commitdate": "1992-09-28", "l_receiptdate": "1992-12-02", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans cajole slyly quickly ironic " }
-{ "l_orderkey": 1089, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33251.75, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-08-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly express deposits haggle" }
-{ "l_orderkey": 2432, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28501.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " requests wake alongside of" }
-{ "l_orderkey": 2562, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-15", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar pinto beans. blithely ev" }
-{ "l_orderkey": 3522, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-29", "l_commitdate": "1994-12-15", "l_receiptdate": "1994-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ic tithes. car" }
-{ "l_orderkey": 3523, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22801.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke according to the doggedly re" }
-{ "l_orderkey": 3943, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully regular deposits accord" }
-{ "l_orderkey": 4581, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6650.35, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express accounts d" }
-{ "l_orderkey": 5573, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1900.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " even foxes. specia" }
-{ "l_orderkey": 644, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21851.15, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uctions nag quickly alongside of t" }
-{ "l_orderkey": 1124, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23751.25, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-10-14", "l_receiptdate": "1998-08-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ggle slyly according" }
-{ "l_orderkey": 1762, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37051.95, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-11-09", "l_receiptdate": "1994-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic platelets sleep along t" }
-{ "l_orderkey": 2724, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20901.1, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-10-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "express fo" }
-{ "l_orderkey": 2786, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40852.15, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ons. theodolites after" }
-{ "l_orderkey": 3458, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43702.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-08", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nod across the boldly even instruct" }
-{ "l_orderkey": 3841, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8550.45, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-12-26", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s according to the courts shall nag s" }
-{ "l_orderkey": 4324, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-10-08", "l_receiptdate": "1995-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " express ideas. blithely blit" }
-{ "l_orderkey": 4706, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans. finally special instruct" }
-{ "l_orderkey": 4865, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19951.05, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-10", "l_receiptdate": "1997-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "eposits detect sly" }
-{ "l_orderkey": 5347, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17100.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he ideas among the requests " }
-{ "l_orderkey": 354, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quickly regular grouches will eat. careful" }
-{ "l_orderkey": 1281, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3800.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ggle against the even requests. requests " }
-{ "l_orderkey": 2406, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15200.8, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-11-28", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " special accou" }
-{ "l_orderkey": 2753, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-08", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " carefully bold deposits sublate s" }
-{ "l_orderkey": 3877, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11400.6, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-08-09", "l_receiptdate": "1993-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal requests. even requests are. pac" }
-{ "l_orderkey": 4034, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fully around the furiously ironic re" }
-{ "l_orderkey": 4645, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42752.25, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ular ideas. slyly" }
-{ "l_orderkey": 4967, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14250.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. blithel" }
-{ "l_orderkey": 5249, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29451.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "f the excuses. furiously fin" }
-{ "l_orderkey": 5410, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7600.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-12", "l_commitdate": "1998-10-22", "l_receiptdate": "1998-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly. fluffily ironic platelets alon" }
-{ "l_orderkey": 5926, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic requests" }
+{ "l_orderkey": 4193, "l_partkey": 51, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27580.45, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-03-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. final packages use blit" }
+{ "l_orderkey": 4324, "l_partkey": 51, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41846.2, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-11-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ainst the u" }
+{ "l_orderkey": 4354, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1902.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-09", "l_commitdate": "1994-12-15", "l_receiptdate": "1995-01-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s nag quickly " }
+{ "l_orderkey": 4544, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-20", "l_commitdate": "1997-09-07", "l_receiptdate": "1997-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular packages. s" }
+{ "l_orderkey": 4741, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 37090.95, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-28", "l_commitdate": "1992-10-03", "l_receiptdate": "1992-11-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "t, regular requests" }
+{ "l_orderkey": 4836, "l_partkey": 51, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11412.6, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-10", "l_receiptdate": "1997-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "sly ironic accoun" }
+{ "l_orderkey": 4932, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12363.65, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-10-16", "l_receiptdate": "1993-09-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "slyly according to the furiously fin" }
+{ "l_orderkey": 5088, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38993.05, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-22", "l_commitdate": "1993-03-07", "l_receiptdate": "1993-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ing requests. " }
+{ "l_orderkey": 5860, "l_partkey": 51, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9510.5, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-03-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ual patterns try to eat carefully above" }
 { "l_orderkey": 97, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 35151.85, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-04-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic requests boost carefully quic" }
+{ "l_orderkey": 354, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-06-03", "l_receiptdate": "1996-05-08", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quickly regular grouches will eat. careful" }
+{ "l_orderkey": 644, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21851.15, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uctions nag quickly alongside of t" }
 { "l_orderkey": 870, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34201.8, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-18", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-11-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "fily. furiously final accounts are " }
+{ "l_orderkey": 1089, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33251.75, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-14", "l_commitdate": "1996-07-10", "l_receiptdate": "1996-08-26", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly express deposits haggle" }
+{ "l_orderkey": 1124, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 23751.25, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-10-14", "l_receiptdate": "1998-08-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ggle slyly according" }
+{ "l_orderkey": 1281, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3800.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-02-21", "l_receiptdate": "1995-03-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ggle against the even requests. requests " }
 { "l_orderkey": 1475, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11400.6, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1997-12-30", "l_receiptdate": "1998-01-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "arefully-- excuses sublate" }
 { "l_orderkey": 1506, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 36101.9, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-12-19", "l_receiptdate": "1992-12-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "xpress, regular excuse" }
+{ "l_orderkey": 1762, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 37051.95, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-12", "l_commitdate": "1994-11-09", "l_receiptdate": "1994-10-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " ironic platelets sleep along t" }
+{ "l_orderkey": 2406, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15200.8, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-11-28", "l_receiptdate": "1996-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " special accou" }
+{ "l_orderkey": 2432, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28501.5, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " requests wake alongside of" }
+{ "l_orderkey": 2562, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-15", "l_commitdate": "1992-10-08", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar pinto beans. blithely ev" }
+{ "l_orderkey": 2724, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20901.1, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-19", "l_commitdate": "1994-11-18", "l_receiptdate": "1994-10-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "express fo" }
+{ "l_orderkey": 2753, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-08", "l_commitdate": "1994-01-17", "l_receiptdate": "1994-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " carefully bold deposits sublate s" }
+{ "l_orderkey": 2786, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40852.15, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-22", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-04-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ons. theodolites after" }
 { "l_orderkey": 2885, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 40.0, "l_extendedprice": 38002.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-23", "l_commitdate": "1992-11-15", "l_receiptdate": "1992-10-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " express depos" }
+{ "l_orderkey": 3458, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43702.3, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-08", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nod across the boldly even instruct" }
+{ "l_orderkey": 3522, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-29", "l_commitdate": "1994-12-15", "l_receiptdate": "1994-12-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ic tithes. car" }
+{ "l_orderkey": 3523, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22801.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-02", "l_commitdate": "1998-06-22", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ke according to the doggedly re" }
+{ "l_orderkey": 3841, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8550.45, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-12-26", "l_receiptdate": "1994-11-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "s according to the courts shall nag s" }
+{ "l_orderkey": 3877, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11400.6, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-08-09", "l_receiptdate": "1993-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal requests. even requests are. pac" }
+{ "l_orderkey": 3943, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-11-10", "l_receiptdate": "1997-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "arefully regular deposits accord" }
+{ "l_orderkey": 4034, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4750.25, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fully around the furiously ironic re" }
 { "l_orderkey": 4131, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5700.3, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-27", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ns cajole slyly. even, iro" }
+{ "l_orderkey": 4324, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13300.7, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-10-08", "l_receiptdate": "1995-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " express ideas. blithely blit" }
+{ "l_orderkey": 4581, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6650.35, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-09", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-10-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "express accounts d" }
 { "l_orderkey": 4612, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16150.85, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-09", "l_commitdate": "1993-11-08", "l_receiptdate": "1994-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "equests haggle carefully silent excus" }
+{ "l_orderkey": 4645, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42752.25, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-27", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ular ideas. slyly" }
+{ "l_orderkey": 4706, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-04-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "into beans. finally special instruct" }
+{ "l_orderkey": 4865, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19951.05, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-17", "l_commitdate": "1997-08-10", "l_receiptdate": "1997-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "eposits detect sly" }
+{ "l_orderkey": 4967, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14250.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. blithel" }
 { "l_orderkey": 5031, "l_partkey": 50, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14250.75, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-01", "l_commitdate": "1995-02-24", "l_receiptdate": "1995-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "yly pending theodolites." }
+{ "l_orderkey": 5249, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29451.55, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-21", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-12-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "f the excuses. furiously fin" }
+{ "l_orderkey": 5347, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17100.9, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "he ideas among the requests " }
+{ "l_orderkey": 5410, "l_partkey": 50, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7600.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-12", "l_commitdate": "1998-10-22", "l_receiptdate": "1998-09-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly. fluffily ironic platelets alon" }
+{ "l_orderkey": 5573, "l_partkey": 50, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1900.1, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-26", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-09-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " even foxes. specia" }
 { "l_orderkey": 5925, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 45602.4, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1996-01-19", "l_receiptdate": "1996-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " haggle after the fo" }
+{ "l_orderkey": 5926, "l_partkey": 50, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25651.35, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ironic requests" }
+{ "l_orderkey": 455, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42706.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-20", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "thrash ironically regular packages. qui" }
 { "l_orderkey": 739, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11388.48, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-20", "l_commitdate": "1998-07-24", "l_receiptdate": "1998-08-22", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "le slyly along the close i" }
+{ "l_orderkey": 768, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 31318.32, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sly ironic instructions. excuses can hagg" }
 { "l_orderkey": 898, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10439.44, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-13", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "etly bold accounts " }
+{ "l_orderkey": 933, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21827.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the furiously bold dinos. sly" }
+{ "l_orderkey": 1157, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15184.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-09", "l_receiptdate": "1998-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tions hang" }
+{ "l_orderkey": 1191, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27522.16, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular pin" }
+{ "l_orderkey": 1220, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23726.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages affi" }
 { "l_orderkey": 1286, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45553.92, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-11", "l_commitdate": "1993-07-11", "l_receiptdate": "1993-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "unts alongs" }
 { "l_orderkey": 1543, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2847.12, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-29", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "sleep along the furiou" }
+{ "l_orderkey": 1569, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-31", "l_receiptdate": "1998-06-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " instructions." }
+{ "l_orderkey": 1761, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 35114.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular packages wake after" }
 { "l_orderkey": 2022, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45553.92, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-07-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "counts. slyly enticing accounts are during " }
 { "l_orderkey": 2050, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27522.16, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-23", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-10-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "oxes alongsid" }
-{ "l_orderkey": 2464, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9490.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-04", "l_commitdate": "1997-12-29", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "slyly final pinto bean" }
-{ "l_orderkey": 3683, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38910.64, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ress instructions. slyly express a" }
-{ "l_orderkey": 5157, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11388.48, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-07", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es. busily " }
-{ "l_orderkey": 1157, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15184.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-09", "l_receiptdate": "1998-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tions hang" }
-{ "l_orderkey": 1220, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23726.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "packages affi" }
-{ "l_orderkey": 1761, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 35114.48, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-03-12", "l_receiptdate": "1994-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "regular packages wake after" }
 { "l_orderkey": 2115, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44604.88, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-29", "l_commitdate": "1998-07-30", "l_receiptdate": "1998-09-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "regular accounts integrate brav" }
+{ "l_orderkey": 2149, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 44604.88, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely final depo" }
+{ "l_orderkey": 2435, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "alongside of the s" }
+{ "l_orderkey": 2464, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9490.4, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-04", "l_commitdate": "1997-12-29", "l_receiptdate": "1998-02-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "slyly final pinto bean" }
+{ "l_orderkey": 2983, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10439.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-02-27", "l_receiptdate": "1992-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "aids integrate s" }
 { "l_orderkey": 3396, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "cial packages cajole blithely around the " }
 { "l_orderkey": 3426, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 29420.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-11", "l_commitdate": "1996-12-10", "l_receiptdate": "1996-12-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " even sentiment" }
-{ "l_orderkey": 4771, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8541.36, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-19", "l_receiptdate": "1993-03-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "riously after the packages. fina" }
-{ "l_orderkey": 768, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 33.0, "l_extendedprice": 31318.32, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-09-29", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sly ironic instructions. excuses can hagg" }
-{ "l_orderkey": 933, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21827.92, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-13", "l_commitdate": "1992-09-18", "l_receiptdate": "1992-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the furiously bold dinos. sly" }
-{ "l_orderkey": 1191, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27522.16, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-28", "l_receiptdate": "1996-02-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular pin" }
-{ "l_orderkey": 2149, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 44604.88, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-27", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "hely final depo" }
-{ "l_orderkey": 4486, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18031.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-28", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pending foxes after" }
-{ "l_orderkey": 455, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42706.8, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-20", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "thrash ironically regular packages. qui" }
-{ "l_orderkey": 1569, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-31", "l_receiptdate": "1998-06-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " instructions." }
-{ "l_orderkey": 2435, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40808.72, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-27", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "alongside of the s" }
-{ "l_orderkey": 2983, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10439.44, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-02-27", "l_receiptdate": "1992-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "aids integrate s" }
 { "l_orderkey": 3653, "l_partkey": 49, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1898.08, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-02", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-06-29", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "n accounts. fina" }
+{ "l_orderkey": 3683, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38910.64, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-26", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ress instructions. slyly express a" }
 { "l_orderkey": 4225, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23726.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-10", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "se fluffily. busily ironic requests are;" }
+{ "l_orderkey": 4486, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 18031.76, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-28", "l_receiptdate": "1998-07-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "pending foxes after" }
 { "l_orderkey": 4711, "l_partkey": 49, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 15.0, "l_extendedprice": 14235.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-03", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ld requests: furiously final inst" }
-{ "l_orderkey": 997, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16116.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle quickly furiously" }
-{ "l_orderkey": 3104, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44557.88, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-25", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ily daring acc" }
-{ "l_orderkey": 3619, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43609.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "press, expres" }
-{ "l_orderkey": 4192, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45505.92, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-07-11", "l_receiptdate": "1998-09-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ests. quickly bol" }
-{ "l_orderkey": 4324, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11376.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-10-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c packages. furiously express sauternes" }
-{ "l_orderkey": 4422, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " theodolites shal" }
-{ "l_orderkey": 5155, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 948.04, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "oze slyly after the silent, regular idea" }
-{ "l_orderkey": 5473, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " excuses sleep blithely! regular dep" }
+{ "l_orderkey": 4771, "l_partkey": 49, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8541.36, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-02-19", "l_receiptdate": "1993-03-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "riously after the packages. fina" }
+{ "l_orderkey": 5157, "l_partkey": 49, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 12.0, "l_extendedprice": 11388.48, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-19", "l_commitdate": "1997-08-07", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "es. busily " }
+{ "l_orderkey": 420, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42661.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " final accounts. furiously express forges" }
 { "l_orderkey": 832, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22752.96, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-14", "l_receiptdate": "1992-06-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ully. carefully speci" }
+{ "l_orderkey": 928, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22752.96, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s the furiously regular warthogs im" }
+{ "l_orderkey": 992, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "fily. quickly special deposit" }
+{ "l_orderkey": 997, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16116.68, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-28", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle quickly furiously" }
+{ "l_orderkey": 1157, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7584.32, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-03-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely even pa" }
 { "l_orderkey": 1218, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41713.76, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-05", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "thely ironic accounts wake slyly" }
 { "l_orderkey": 1574, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-08", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s. slyly regular depen" }
-{ "l_orderkey": 2304, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l excuses after the ev" }
-{ "l_orderkey": 2503, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s cajole. slyly close courts nod f" }
-{ "l_orderkey": 2753, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37921.6, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "latelets kindle slyly final depos" }
-{ "l_orderkey": 3585, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31285.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-14", "l_commitdate": "1995-01-19", "l_receiptdate": "1994-12-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ironic dependencies serve furi" }
-{ "l_orderkey": 4870, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46453.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-14", "l_commitdate": "1994-10-24", "l_receiptdate": "1994-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular packages " }
-{ "l_orderkey": 992, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-13", "l_commitdate": "1997-12-28", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "fily. quickly special deposit" }
-{ "l_orderkey": 1831, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ent deposits. regular saute" }
-{ "l_orderkey": 2211, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23701.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas. carefully special theodolites along" }
-{ "l_orderkey": 3042, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18012.76, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-05", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e carefully. regul" }
-{ "l_orderkey": 3937, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28441.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-17", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al packages slee" }
-{ "l_orderkey": 4836, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15168.64, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-03-05", "l_receiptdate": "1997-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gular packages against the express reque" }
-{ "l_orderkey": 5474, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29389.24, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously express ideas. speci" }
-{ "l_orderkey": 420, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42661.8, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-14", "l_commitdate": "1996-01-01", "l_receiptdate": "1996-01-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " final accounts. furiously express forges" }
-{ "l_orderkey": 928, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22752.96, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s the furiously regular warthogs im" }
-{ "l_orderkey": 1157, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7584.32, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-03-16", "l_receiptdate": "1998-03-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "blithely even pa" }
 { "l_orderkey": 1634, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "counts alo" }
 { "l_orderkey": 1667, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5688.24, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-21", "l_commitdate": "1997-12-19", "l_receiptdate": "1998-01-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " nag quickly above th" }
 { "l_orderkey": 1793, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27493.16, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-09-20", "l_receiptdate": "1992-11-23", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ar excuses. " }
+{ "l_orderkey": 1831, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-22", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-04-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ent deposits. regular saute" }
 { "l_orderkey": 2050, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 25.0, "l_extendedprice": 23701.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-18", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y according to " }
+{ "l_orderkey": 2211, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23701.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-09", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-11-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deas. carefully special theodolites along" }
 { "l_orderkey": 2213, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1993-03-31", "l_receiptdate": "1993-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully pend" }
+{ "l_orderkey": 2304, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-03-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l excuses after the ev" }
+{ "l_orderkey": 2503, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2844.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s cajole. slyly close courts nod f" }
 { "l_orderkey": 2660, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 16116.68, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-18", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "al pinto beans wake after the furious" }
 { "l_orderkey": 2691, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1896.08, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-10", "l_commitdate": "1992-06-04", "l_receiptdate": "1992-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s cajole at the blithely ironic warthog" }
+{ "l_orderkey": 2753, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37921.6, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-06", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "latelets kindle slyly final depos" }
 { "l_orderkey": 2785, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 32233.36, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-09-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "kages wake carefully silent " }
+{ "l_orderkey": 3042, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 18012.76, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-05", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-03-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e carefully. regul" }
+{ "l_orderkey": 3104, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 44557.88, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-25", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ily daring acc" }
+{ "l_orderkey": 3585, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31285.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-14", "l_commitdate": "1995-01-19", "l_receiptdate": "1994-12-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ironic dependencies serve furi" }
+{ "l_orderkey": 3619, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43609.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "press, expres" }
+{ "l_orderkey": 3937, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28441.2, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-17", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "al packages slee" }
+{ "l_orderkey": 4192, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 48.0, "l_extendedprice": 45505.92, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-17", "l_commitdate": "1998-07-11", "l_receiptdate": "1998-09-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ests. quickly bol" }
+{ "l_orderkey": 4324, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11376.48, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-05", "l_commitdate": "1995-09-07", "l_receiptdate": "1995-10-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c packages. furiously express sauternes" }
+{ "l_orderkey": 4422, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38869.64, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-12", "l_commitdate": "1995-07-09", "l_receiptdate": "1995-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " theodolites shal" }
+{ "l_orderkey": 4836, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15168.64, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-03-05", "l_receiptdate": "1997-01-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gular packages against the express reque" }
+{ "l_orderkey": 4870, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46453.96, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-14", "l_commitdate": "1994-10-24", "l_receiptdate": "1994-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular packages " }
 { "l_orderkey": 5056, "l_partkey": 48, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6636.28, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-28", "l_commitdate": "1997-04-07", "l_receiptdate": "1997-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "rouches after the pending instruc" }
 { "l_orderkey": 5090, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 19908.84, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-29", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-04-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly express accounts. slyly even r" }
+{ "l_orderkey": 5155, "l_partkey": 48, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 948.04, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-08-11", "l_receiptdate": "1994-07-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "oze slyly after the silent, regular idea" }
+{ "l_orderkey": 5473, "l_partkey": 48, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8532.36, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-03", "l_commitdate": "1992-05-30", "l_receiptdate": "1992-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " excuses sleep blithely! regular dep" }
+{ "l_orderkey": 5474, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29389.24, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the furiously express ideas. speci" }
 { "l_orderkey": 5476, "l_partkey": 48, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12324.52, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-27", "l_commitdate": "1997-12-08", "l_receiptdate": "1997-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special ac" }
+{ "l_orderkey": 100, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43563.84, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-10", "l_receiptdate": "1998-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ular accounts. even" }
+{ "l_orderkey": 289, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts. quickly bold deposits alongside" }
 { "l_orderkey": 486, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 43563.84, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-18", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-04-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "theodolites eat carefully furious" }
 { "l_orderkey": 519, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25570.08, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-20", "l_commitdate": "1997-12-06", "l_receiptdate": "1997-12-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "le. even, final dependencies" }
-{ "l_orderkey": 3105, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28411.2, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-03-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts boost among t" }
-{ "l_orderkey": 3682, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16099.68, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", ironic packages wake a" }
-{ "l_orderkey": 4194, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17046.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1994-12-04", "l_receiptdate": "1995-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ld packages. quickly eve" }
-{ "l_orderkey": 4769, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-08-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". slyly even deposit" }
-{ "l_orderkey": 5761, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38828.64, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "pecial deposits. qu" }
-{ "l_orderkey": 289, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-08", "l_commitdate": "1997-04-06", "l_receiptdate": "1997-06-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ts. quickly bold deposits alongside" }
-{ "l_orderkey": 869, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ong the furiously bold instructi" }
-{ "l_orderkey": 1665, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely final requests. requests" }
-{ "l_orderkey": 2341, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11364.48, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-06", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". quickly final deposits sl" }
-{ "l_orderkey": 2465, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 47352.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the pending th" }
-{ "l_orderkey": 3685, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35040.48, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ress attai" }
-{ "l_orderkey": 5639, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10417.44, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the unusual pinto beans caj" }
-{ "l_orderkey": 100, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43563.84, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-04-10", "l_receiptdate": "1998-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ular accounts. even" }
 { "l_orderkey": 768, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 44510.88, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-28", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "foxes. slyly ironic deposits a" }
 { "l_orderkey": 805, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11364.48, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-13", "l_commitdate": "1995-09-27", "l_receiptdate": "1995-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " regular foxes. furio" }
+{ "l_orderkey": 869, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-03-17", "l_receiptdate": "1997-05-24", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ong the furiously bold instructi" }
+{ "l_orderkey": 899, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23676.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rly final sentiments. bold pinto beans " }
 { "l_orderkey": 1156, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 20.0, "l_extendedprice": 18940.8, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-01", "l_commitdate": "1997-01-06", "l_receiptdate": "1997-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "deposits sleep bravel" }
 { "l_orderkey": 1184, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25570.08, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-10", "l_commitdate": "1997-12-02", "l_receiptdate": "1998-02-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s wake fluffily. fl" }
-{ "l_orderkey": 2375, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24623.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "rate across the" }
-{ "l_orderkey": 3171, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 32199.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r the final, even packages. quickly" }
-{ "l_orderkey": 4037, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-12", "l_receiptdate": "1993-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s around the blithely ironic ac" }
-{ "l_orderkey": 4387, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8523.36, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-04", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c ideas. slyly regular packages sol" }
-{ "l_orderkey": 4609, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26517.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously. quickly final requests cajole fl" }
-{ "l_orderkey": 5636, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-11", "l_commitdate": "1995-04-27", "l_receiptdate": "1995-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en, fluffy accounts amon" }
-{ "l_orderkey": 5637, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13258.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits wak" }
-{ "l_orderkey": 899, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 23676.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-21", "l_commitdate": "1998-05-12", "l_receiptdate": "1998-08-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "rly final sentiments. bold pinto beans " }
+{ "l_orderkey": 1665, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-07", "l_receiptdate": "1994-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely final requests. requests" }
 { "l_orderkey": 2311, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 947.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-06-20", "l_receiptdate": "1995-06-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ptotes. furiously regular theodolite" }
+{ "l_orderkey": 2341, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11364.48, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-06", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": ". quickly final deposits sl" }
+{ "l_orderkey": 2375, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24623.04, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-18", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "rate across the" }
+{ "l_orderkey": 2465, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 47352.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "the pending th" }
+{ "l_orderkey": 3105, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28411.2, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-03", "l_commitdate": "1997-02-03", "l_receiptdate": "1997-03-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ess accounts boost among t" }
+{ "l_orderkey": 3171, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 32199.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-06-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "r the final, even packages. quickly" }
+{ "l_orderkey": 3682, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 16099.68, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-12", "l_commitdate": "1997-04-04", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ", ironic packages wake a" }
+{ "l_orderkey": 3685, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 35040.48, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-09", "l_receiptdate": "1992-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ress attai" }
+{ "l_orderkey": 4037, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3788.16, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-06-12", "l_receiptdate": "1993-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "s around the blithely ironic ac" }
+{ "l_orderkey": 4194, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 17046.72, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-14", "l_commitdate": "1994-12-04", "l_receiptdate": "1995-03-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ld packages. quickly eve" }
+{ "l_orderkey": 4387, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8523.36, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-04", "l_commitdate": "1995-12-26", "l_receiptdate": "1996-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "c ideas. slyly regular packages sol" }
 { "l_orderkey": 4421, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 41669.76, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-17", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-06-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "le carefully. bl" }
 { "l_orderkey": 4548, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 16099.68, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-23", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y ironic requests above the fluffily d" }
 { "l_orderkey": 4608, "l_partkey": 47, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47352.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-25", "l_commitdate": "1994-09-01", "l_receiptdate": "1994-08-10", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " theodolites" }
-{ "l_orderkey": 166, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1995-11-29", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e carefully bold " }
-{ "l_orderkey": 901, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1892.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-25", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d foxes use slyly" }
-{ "l_orderkey": 2560, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29327.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-14", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "to beans. blithely regular Tiresias int" }
-{ "l_orderkey": 3043, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21758.92, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uickly above the pending," }
-{ "l_orderkey": 3201, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-27", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-10-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing to the furiously expr" }
-{ "l_orderkey": 3840, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-10-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress pinto beans. accounts a" }
-{ "l_orderkey": 4320, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26489.12, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "nts. even, ironic excuses hagg" }
-{ "l_orderkey": 4711, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17028.72, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " blithely. bold asymptote" }
-{ "l_orderkey": 1318, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24597.04, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-26", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly. regular, u" }
-{ "l_orderkey": 1859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-06-05", "l_receiptdate": "1997-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ffily ironic pac" }
-{ "l_orderkey": 2503, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47302.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s around the slyly " }
-{ "l_orderkey": 2919, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41625.76, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final ideas haggle carefully fluff" }
-{ "l_orderkey": 3173, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15136.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-12", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e special," }
-{ "l_orderkey": 3525, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar excuses wake carefull" }
-{ "l_orderkey": 4514, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14190.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "! unusual, special deposits afte" }
-{ "l_orderkey": 5633, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25543.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-28", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ructions. even ideas haggle carefully r" }
-{ "l_orderkey": 5767, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34057.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ake carefully. packages " }
+{ "l_orderkey": 4609, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26517.12, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously. quickly final requests cajole fl" }
+{ "l_orderkey": 4769, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 34093.44, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-22", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-08-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": ". slyly even deposit" }
+{ "l_orderkey": 5636, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12311.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-11", "l_commitdate": "1995-04-27", "l_receiptdate": "1995-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "en, fluffy accounts amon" }
+{ "l_orderkey": 5637, "l_partkey": 47, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13258.56, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y bold deposits wak" }
+{ "l_orderkey": 5639, "l_partkey": 47, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10417.44, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "g the unusual pinto beans caj" }
+{ "l_orderkey": 5761, "l_partkey": 47, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38828.64, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-31", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "pecial deposits. qu" }
 { "l_orderkey": 70, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-03-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "alongside of the deposits. fur" }
+{ "l_orderkey": 166, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-30", "l_commitdate": "1995-11-29", "l_receiptdate": "1996-01-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e carefully bold " }
+{ "l_orderkey": 356, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3784.16, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the dependencies nod unusual, final ac" }
+{ "l_orderkey": 901, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1892.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-25", "l_commitdate": "1998-09-27", "l_receiptdate": "1998-11-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "d foxes use slyly" }
 { "l_orderkey": 1031, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14190.6, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-07", "l_commitdate": "1994-10-29", "l_receiptdate": "1994-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "about the carefully bold a" }
 { "l_orderkey": 1120, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20812.88, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-15", "l_commitdate": "1998-01-25", "l_receiptdate": "1997-12-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ons. slyly silent requests sleep silent" }
+{ "l_orderkey": 1318, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24597.04, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-26", "l_commitdate": "1998-08-09", "l_receiptdate": "1998-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ly. regular, u" }
+{ "l_orderkey": 1859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-06-05", "l_receiptdate": "1997-07-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ffily ironic pac" }
+{ "l_orderkey": 2370, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2838.12, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly regular Tiresia" }
 { "l_orderkey": 2435, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 16082.68, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-05", "l_commitdate": "1993-05-05", "l_receiptdate": "1993-06-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cajole aft" }
+{ "l_orderkey": 2499, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45409.92, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic ideas cajole quickly requests. caref" }
+{ "l_orderkey": 2503, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 47302.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s around the slyly " }
+{ "l_orderkey": 2560, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29327.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-14", "l_commitdate": "1992-10-14", "l_receiptdate": "1992-12-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "to beans. blithely regular Tiresias int" }
+{ "l_orderkey": 2919, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41625.76, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-01-12", "l_receiptdate": "1994-04-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "final ideas haggle carefully fluff" }
+{ "l_orderkey": 3043, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21758.92, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-08", "l_commitdate": "1992-07-22", "l_receiptdate": "1992-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uickly above the pending," }
 { "l_orderkey": 3141, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44463.88, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1996-01-13", "l_receiptdate": "1995-12-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " are slyly pi" }
+{ "l_orderkey": 3173, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15136.64, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-12", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e special," }
+{ "l_orderkey": 3201, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10406.44, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-27", "l_commitdate": "1993-08-29", "l_receiptdate": "1993-10-18", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing to the furiously expr" }
+{ "l_orderkey": 3525, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-08", "l_commitdate": "1996-03-18", "l_receiptdate": "1996-03-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar excuses wake carefull" }
 { "l_orderkey": 3648, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32165.36, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-21", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " deposits are furiously. careful, " }
 { "l_orderkey": 3684, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5676.24, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-09", "l_commitdate": "1993-10-05", "l_receiptdate": "1993-09-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "he silent requests. packages sleep fu" }
-{ "l_orderkey": 3845, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 946.04, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " blithely ironic t" }
-{ "l_orderkey": 4676, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-10-18", "l_receiptdate": "1996-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "cuses boost above" }
-{ "l_orderkey": 5859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31219.32, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eposits unwind furiously final pinto bea" }
-{ "l_orderkey": 356, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3784.16, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-28", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " the dependencies nod unusual, final ac" }
-{ "l_orderkey": 2370, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2838.12, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-03-26", "l_receiptdate": "1994-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly regular Tiresia" }
-{ "l_orderkey": 2499, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45409.92, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ronic ideas cajole quickly requests. caref" }
 { "l_orderkey": 3779, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26489.12, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-06", "l_commitdate": "1997-04-01", "l_receiptdate": "1997-05-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s. close requests sleep" }
+{ "l_orderkey": 3840, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11352.48, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-10-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "xpress pinto beans. accounts a" }
+{ "l_orderkey": 3845, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 946.04, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-07", "l_receiptdate": "1992-06-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " blithely ironic t" }
 { "l_orderkey": 4230, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35949.52, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-04-21", "l_receiptdate": "1992-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly regular packages. regular ideas boost" }
+{ "l_orderkey": 4320, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26489.12, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-02-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "nts. even, ironic excuses hagg" }
 { "l_orderkey": 4322, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 16082.68, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-05-31", "l_receiptdate": "1998-06-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ructions boost " }
+{ "l_orderkey": 4514, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14190.6, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "! unusual, special deposits afte" }
+{ "l_orderkey": 4676, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7568.32, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-10-18", "l_receiptdate": "1996-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "cuses boost above" }
+{ "l_orderkey": 4711, "l_partkey": 46, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 18.0, "l_extendedprice": 17028.72, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-07-31", "l_receiptdate": "1998-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " blithely. bold asymptote" }
+{ "l_orderkey": 5633, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25543.08, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-28", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ructions. even ideas haggle carefully r" }
 { "l_orderkey": 5665, "l_partkey": 46, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44463.88, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-06", "l_commitdate": "1993-09-19", "l_receiptdate": "1993-11-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s mold fluffily. final deposits along the" }
+{ "l_orderkey": 5767, "l_partkey": 46, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34057.44, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ake carefully. packages " }
+{ "l_orderkey": 5859, "l_partkey": 46, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31219.32, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-06-22", "l_receiptdate": "1997-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "eposits unwind furiously final pinto bea" }
+{ "l_orderkey": 32, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1890.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " express accounts wake according to the" }
+{ "l_orderkey": 98, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13230.56, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1994-11-22", "l_receiptdate": "1995-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " cajole furiously. blithely ironic ideas " }
+{ "l_orderkey": 131, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ending requests. final, ironic pearls slee" }
 { "l_orderkey": 417, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 38746.64, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-11", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "tes. regular requests across the " }
 { "l_orderkey": 517, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26461.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-05-18", "l_receiptdate": "1997-05-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " requests. special, fi" }
+{ "l_orderkey": 643, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 36856.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " the pains. carefully s" }
+{ "l_orderkey": 930, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-21", "l_commitdate": "1995-02-20", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quickly regular pinto beans sle" }
+{ "l_orderkey": 1158, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4725.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "symptotes along the care" }
+{ "l_orderkey": 1893, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2835.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gular, even ideas. fluffily bol" }
 { "l_orderkey": 2055, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14175.6, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-15", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-10-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "furiously bold " }
+{ "l_orderkey": 2278, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic pinto beans br" }
 { "l_orderkey": 2278, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-09", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "into beans. blit" }
 { "l_orderkey": 2337, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 46306.96, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-08-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " along the packages. furiously p" }
 { "l_orderkey": 2720, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4725.2, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-24", "l_commitdate": "1993-08-08", "l_receiptdate": "1993-07-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ously ironic foxes thrash" }
+{ "l_orderkey": 2818, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10395.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ggle across the carefully blithe" }
+{ "l_orderkey": 3009, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45361.92, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " dependencies sleep quickly a" }
 { "l_orderkey": 3105, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8505.36, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-02-04", "l_receiptdate": "1997-01-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "es wake among t" }
+{ "l_orderkey": 3239, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-09", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-02-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "d blithely stea" }
 { "l_orderkey": 3239, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40636.72, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "y. bold pinto beans use " }
+{ "l_orderkey": 3460, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 26461.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "inal, ironic instructions. carefully" }
+{ "l_orderkey": 3686, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29296.24, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-09", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle across the courts. furiously regu" }
+{ "l_orderkey": 3879, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33076.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-23", "l_receiptdate": "1995-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "o beans. accounts cajole furiously. re" }
+{ "l_orderkey": 4515, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20790.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le quickly above the even, bold ideas." }
 { "l_orderkey": 4518, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17955.76, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-09", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ter the slyly bo" }
 { "l_orderkey": 4547, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14175.6, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-12-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "e carefully across the unus" }
+{ "l_orderkey": 4548, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tions integrat" }
 { "l_orderkey": 4935, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 46306.96, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-16", "l_commitdate": "1993-08-21", "l_receiptdate": "1993-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ffily after the furiou" }
 { "l_orderkey": 4960, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5670.24, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-21", "l_commitdate": "1995-05-13", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ual package" }
-{ "l_orderkey": 5507, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3780.16, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "into beans are" }
-{ "l_orderkey": 98, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13230.56, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-30", "l_commitdate": "1994-11-22", "l_receiptdate": "1995-01-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " cajole furiously. blithely ironic ideas " }
-{ "l_orderkey": 1158, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4725.2, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-07-30", "l_receiptdate": "1996-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "symptotes along the care" }
-{ "l_orderkey": 2278, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-04", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-30", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic pinto beans br" }
-{ "l_orderkey": 3879, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33076.4, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-08", "l_commitdate": "1996-01-23", "l_receiptdate": "1995-12-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "o beans. accounts cajole furiously. re" }
-{ "l_orderkey": 4548, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-12", "l_receiptdate": "1996-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "tions integrat" }
-{ "l_orderkey": 5158, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 40636.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual platelets. slyly even foxes cajole " }
-{ "l_orderkey": 32, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1890.08, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-07", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " express accounts wake according to the" }
-{ "l_orderkey": 131, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-10", "l_receiptdate": "1994-09-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ending requests. final, ironic pearls slee" }
-{ "l_orderkey": 643, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 36856.56, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " the pains. carefully s" }
-{ "l_orderkey": 1893, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2835.12, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "gular, even ideas. fluffily bol" }
-{ "l_orderkey": 3009, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 45361.92, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-19", "l_commitdate": "1997-05-13", "l_receiptdate": "1997-04-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " dependencies sleep quickly a" }
-{ "l_orderkey": 3460, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 28.0, "l_extendedprice": 26461.12, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-28", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-11-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "inal, ironic instructions. carefully" }
-{ "l_orderkey": 4515, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20790.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-16", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-07-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le quickly above the even, bold ideas." }
-{ "l_orderkey": 930, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 34021.44, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-21", "l_commitdate": "1995-02-20", "l_receiptdate": "1994-12-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quickly regular pinto beans sle" }
-{ "l_orderkey": 2818, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10395.44, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-18", "l_commitdate": "1995-02-11", "l_receiptdate": "1995-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ggle across the carefully blithe" }
-{ "l_orderkey": 3239, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47252.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-09", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-02-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "d blithely stea" }
-{ "l_orderkey": 3686, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 29296.24, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-09", "l_commitdate": "1998-08-28", "l_receiptdate": "1998-10-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "gle across the courts. furiously regu" }
 { "l_orderkey": 5092, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 32131.36, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-26", "l_receiptdate": "1995-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ckages nag " }
 { "l_orderkey": 5122, "l_partkey": 45, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 11340.48, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-02", "l_commitdate": "1996-04-27", "l_receiptdate": "1996-04-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lar instructions " }
+{ "l_orderkey": 5158, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 40636.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "nusual platelets. slyly even foxes cajole " }
 { "l_orderkey": 5223, "l_partkey": 45, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22680.96, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "refully bold courts besides the regular," }
+{ "l_orderkey": 5507, "l_partkey": 45, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3780.16, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-06", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "into beans are" }
 { "l_orderkey": 5537, "l_partkey": 45, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9450.4, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-25", "l_receiptdate": "1997-01-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " sleep carefully slyly bold depos" }
-{ "l_orderkey": 739, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the theodolites sn" }
-{ "l_orderkey": 1600, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45313.92, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously silent foxes could wake. car" }
-{ "l_orderkey": 3073, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10384.44, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions sleep according to the " }
-{ "l_orderkey": 3621, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18880.8, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gular accounts use carefully with" }
-{ "l_orderkey": 5284, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22656.96, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle according " }
-{ "l_orderkey": 5728, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-25", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nd the bravely final deposits. final ideas" }
 { "l_orderkey": 322, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45313.92, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-06-16", "l_receiptdate": "1992-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dolites detect qu" }
+{ "l_orderkey": 739, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-08-28", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the theodolites sn" }
 { "l_orderkey": 838, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16992.72, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-28", "l_commitdate": "1998-04-06", "l_receiptdate": "1998-03-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "hely unusual foxes. furio" }
+{ "l_orderkey": 932, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38705.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes. ironic pl" }
 { "l_orderkey": 1024, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26433.12, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-04", "l_commitdate": "1998-03-12", "l_receiptdate": "1998-03-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "e blithely regular pi" }
+{ "l_orderkey": 1153, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-07-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " theodolites" }
 { "l_orderkey": 1350, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30209.28, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-18", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ic, final " }
+{ "l_orderkey": 1600, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 45313.92, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-04-14", "l_receiptdate": "1993-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously silent foxes could wake. car" }
+{ "l_orderkey": 2147, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32097.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "egular deposits hang car" }
+{ "l_orderkey": 3073, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10384.44, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-01", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "instructions sleep according to the " }
 { "l_orderkey": 3138, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25489.08, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "counts cajole fluffily carefully special i" }
+{ "l_orderkey": 3138, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-19", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites around the carefully busy the" }
+{ "l_orderkey": 3269, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 36817.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "he express packages?" }
+{ "l_orderkey": 3621, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18880.8, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-07-04", "l_receiptdate": "1993-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "gular accounts use carefully with" }
+{ "l_orderkey": 4292, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20768.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-02-16", "l_receiptdate": "1992-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "refully expres" }
+{ "l_orderkey": 4961, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35873.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e on the blithely bold accounts. unu" }
 { "l_orderkey": 5250, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1888.08, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-10-10", "l_receiptdate": "1995-08-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its. final pinto" }
+{ "l_orderkey": 5284, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22656.96, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " haggle according " }
 { "l_orderkey": 5319, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36817.56, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-06-11", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "unts. furiously silent" }
 { "l_orderkey": 5381, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 29265.24, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-10", "l_commitdate": "1993-03-22", "l_receiptdate": "1993-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the carefully expre" }
 { "l_orderkey": 5568, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16992.72, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-19", "l_commitdate": "1995-08-18", "l_receiptdate": "1995-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "structions haggle. carefully regular " }
+{ "l_orderkey": 5728, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44369.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-25", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nd the bravely final deposits. final ideas" }
 { "l_orderkey": 5859, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8496.36, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-06-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ges boost quickly. blithely r" }
-{ "l_orderkey": 2147, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 32097.36, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-29", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "egular deposits hang car" }
-{ "l_orderkey": 3138, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-19", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-06-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "dolites around the carefully busy the" }
-{ "l_orderkey": 3269, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 39.0, "l_extendedprice": 36817.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-05-26", "l_receiptdate": "1996-03-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "he express packages?" }
-{ "l_orderkey": 4292, "l_partkey": 44, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20768.88, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-02-16", "l_receiptdate": "1992-03-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "refully expres" }
-{ "l_orderkey": 932, "l_partkey": 44, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38705.64, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-07-22", "l_receiptdate": "1997-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes. ironic pl" }
-{ "l_orderkey": 1153, "l_partkey": 44, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23601.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-07-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " theodolites" }
-{ "l_orderkey": 4961, "l_partkey": 44, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35873.52, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e on the blithely bold accounts. unu" }
 { "l_orderkey": 326, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44322.88, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-16", "l_commitdate": "1995-07-04", "l_receiptdate": "1995-10-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " special accounts sleep " }
-{ "l_orderkey": 1892, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-05-09", "l_receiptdate": "1994-05-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "hes nod furiously around the instruc" }
-{ "l_orderkey": 2372, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39607.68, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-02", "l_receiptdate": "1998-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar packages. regular" }
-{ "l_orderkey": 3333, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42436.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-04", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "dolites. quickly r" }
-{ "l_orderkey": 4259, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13202.56, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1997-11-21", "l_receiptdate": "1998-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously pending excuses. ideas hagg" }
-{ "l_orderkey": 5959, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44322.88, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-07-24", "l_receiptdate": "1992-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deposits. slyly special cou" }
+{ "l_orderkey": 678, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10373.44, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-28", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess deposits dazzle f" }
 { "l_orderkey": 707, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20746.88, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-12", "l_commitdate": "1994-12-28", "l_receiptdate": "1995-01-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " kindle ironically" }
 { "l_orderkey": 901, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 34892.48, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-01", "l_commitdate": "1998-09-13", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ickly final deposits " }
+{ "l_orderkey": 1127, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "l instructions boost blithely according " }
 { "l_orderkey": 1639, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35835.52, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-23", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-08-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y regular packages. b" }
+{ "l_orderkey": 1892, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-05-09", "l_receiptdate": "1994-05-03", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "hes nod furiously around the instruc" }
 { "l_orderkey": 2023, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27348.16, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-29", "l_commitdate": "1992-07-28", "l_receiptdate": "1992-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "usual instructions. bli" }
+{ "l_orderkey": 2052, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15088.64, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-30", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final deposits cajole according " }
+{ "l_orderkey": 2208, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "al foxes will hav" }
 { "l_orderkey": 2371, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 33.0, "l_extendedprice": 31120.32, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-30", "l_commitdate": "1998-02-06", "l_receiptdate": "1998-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deas are. express r" }
+{ "l_orderkey": 2372, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39607.68, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-04", "l_commitdate": "1998-01-02", "l_receiptdate": "1998-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lar packages. regular" }
+{ "l_orderkey": 2659, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19803.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-23", "l_commitdate": "1994-02-10", "l_receiptdate": "1994-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y beyond the furiously even co" }
+{ "l_orderkey": 2978, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24519.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "as haggle against the carefully express dep" }
+{ "l_orderkey": 3333, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 42436.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-04", "l_commitdate": "1992-11-08", "l_receiptdate": "1992-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "dolites. quickly r" }
 { "l_orderkey": 3395, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 40550.72, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-13", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ckages above the furiously regu" }
+{ "l_orderkey": 3555, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23576.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sual packages. quickly " }
+{ "l_orderkey": 3653, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8487.36, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-08-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tes: blithely bo" }
+{ "l_orderkey": 3808, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26405.12, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lly final accounts alo" }
+{ "l_orderkey": 3811, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17917.76, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-07-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s boost blithely furiou" }
+{ "l_orderkey": 4069, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30177.28, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unts. deposit" }
+{ "l_orderkey": 4259, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13202.56, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-09", "l_commitdate": "1997-11-21", "l_receiptdate": "1998-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously pending excuses. ideas hagg" }
+{ "l_orderkey": 4324, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 29234.24, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-23", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "efully flu" }
+{ "l_orderkey": 4517, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-08", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "refully pending acco" }
 { "l_orderkey": 5191, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25462.08, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1995-01-24", "l_receiptdate": "1995-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "tructions nag bravely within the re" }
+{ "l_orderkey": 5281, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31120.32, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly brave foxes. bold deposits above the " }
 { "l_orderkey": 5285, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11316.48, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-22", "l_commitdate": "1994-04-07", "l_receiptdate": "1994-05-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " deposits-- quickly bold requests hag" }
 { "l_orderkey": 5444, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37721.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-09", "l_commitdate": "1995-04-25", "l_receiptdate": "1995-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ously bold ideas. instructions wake slyl" }
-{ "l_orderkey": 5831, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34892.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously even requests" }
-{ "l_orderkey": 678, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10373.44, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-28", "l_commitdate": "1993-05-16", "l_receiptdate": "1993-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ess deposits dazzle f" }
-{ "l_orderkey": 1127, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 33006.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-25", "l_commitdate": "1995-11-03", "l_receiptdate": "1995-12-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "l instructions boost blithely according " }
-{ "l_orderkey": 2052, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 15088.64, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-30", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-07-12", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final deposits cajole according " }
-{ "l_orderkey": 2659, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19803.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-23", "l_commitdate": "1994-02-10", "l_receiptdate": "1994-01-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "y beyond the furiously even co" }
-{ "l_orderkey": 3653, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8487.36, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-08-17", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "tes: blithely bo" }
-{ "l_orderkey": 3811, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17917.76, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-20", "l_commitdate": "1998-06-14", "l_receiptdate": "1998-07-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s boost blithely furiou" }
-{ "l_orderkey": 4517, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-08", "l_commitdate": "1998-04-18", "l_receiptdate": "1998-06-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "refully pending acco" }
 { "l_orderkey": 5793, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7544.32, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "al foxes l" }
-{ "l_orderkey": 2208, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 47152.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-31", "l_receiptdate": "1995-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "al foxes will hav" }
-{ "l_orderkey": 2978, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24519.04, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-22", "l_receiptdate": "1995-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "as haggle against the carefully express dep" }
-{ "l_orderkey": 3555, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23576.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-01", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-10-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sual packages. quickly " }
-{ "l_orderkey": 3808, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26405.12, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-27", "l_commitdate": "1994-06-18", "l_receiptdate": "1994-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lly final accounts alo" }
-{ "l_orderkey": 4069, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 30177.28, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unts. deposit" }
-{ "l_orderkey": 4324, "l_partkey": 43, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 31.0, "l_extendedprice": 29234.24, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-23", "l_commitdate": "1995-09-14", "l_receiptdate": "1995-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "efully flu" }
-{ "l_orderkey": 5281, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 31120.32, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-01", "l_commitdate": "1995-12-28", "l_receiptdate": "1996-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly brave foxes. bold deposits above the " }
+{ "l_orderkey": 5831, "l_partkey": 43, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34892.48, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-02-08", "l_receiptdate": "1997-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously even requests" }
 { "l_orderkey": 5958, "l_partkey": 43, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21689.92, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-26", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "regular requests. bold, bold deposits unwin" }
+{ "l_orderkey": 5959, "l_partkey": 43, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 47.0, "l_extendedprice": 44322.88, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-07-24", "l_receiptdate": "1992-09-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "deposits. slyly special cou" }
+{ "l_orderkey": 197, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-08", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "use slyly slyly silent depo" }
 { "l_orderkey": 260, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25435.08, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-23", "l_commitdate": "1997-02-15", "l_receiptdate": "1997-04-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ions according to the" }
+{ "l_orderkey": 327, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " asymptotes are fu" }
 { "l_orderkey": 771, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6594.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-18", "l_commitdate": "1995-08-31", "l_receiptdate": "1995-06-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "theodolites after the fluffily express " }
 { "l_orderkey": 903, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-22", "l_commitdate": "1995-09-13", "l_receiptdate": "1995-11-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y final platelets sublate among the " }
 { "l_orderkey": 1152, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5652.24, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-07", "l_commitdate": "1994-11-05", "l_receiptdate": "1994-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "p furiously; packages above th" }
-{ "l_orderkey": 1604, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14130.6, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-09-03", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions haggle" }
-{ "l_orderkey": 2560, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits affix quickly. unusual, eve" }
-{ "l_orderkey": 3268, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37681.6, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly. bold, eve" }
-{ "l_orderkey": 3488, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11304.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e slyly; furiously final packages wak" }
-{ "l_orderkey": 4645, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 25435.08, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously express pinto beans. ironic depos" }
-{ "l_orderkey": 5442, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages. accounts haggle dependencies. f" }
-{ "l_orderkey": 197, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-08", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "use slyly slyly silent depo" }
-{ "l_orderkey": 1702, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-31", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ackages sleep. furiously even excuses snooz" }
-{ "l_orderkey": 2310, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45217.92, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-20", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep slyly alongside of the " }
-{ "l_orderkey": 2599, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24493.04, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-21", "l_receiptdate": "1996-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nag carefully " }
-{ "l_orderkey": 4576, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "detect slyly." }
-{ "l_orderkey": 4579, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly across the " }
-{ "l_orderkey": 4994, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22608.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-08-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s. slyly ironic deposits cajole f" }
-{ "l_orderkey": 1413, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5652.24, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lithely excuses. f" }
-{ "l_orderkey": 3585, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 42391.8, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-02-19", "l_receiptdate": "1995-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "are blithely c" }
-{ "l_orderkey": 4837, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ing requests are blithely regular instructi" }
-{ "l_orderkey": 5765, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19782.84, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-05", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ole furiously. quick, special dependencies " }
-{ "l_orderkey": 327, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-24", "l_commitdate": "1995-07-11", "l_receiptdate": "1995-06-05", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " asymptotes are fu" }
 { "l_orderkey": 1216, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16956.72, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-20", "l_commitdate": "1993-01-28", "l_receiptdate": "1993-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final packages nod " }
 { "l_orderkey": 1218, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-09-07", "l_receiptdate": "1994-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "press furio" }
+{ "l_orderkey": 1413, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5652.24, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-09-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "lithely excuses. f" }
 { "l_orderkey": 1571, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 10.0, "l_extendedprice": 9420.4, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-12", "l_commitdate": "1993-02-13", "l_receiptdate": "1992-12-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lets. carefully regular ideas wake" }
+{ "l_orderkey": 1604, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14130.6, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-22", "l_commitdate": "1993-09-03", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions haggle" }
+{ "l_orderkey": 1702, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-07-31", "l_receiptdate": "1995-09-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ackages sleep. furiously even excuses snooz" }
 { "l_orderkey": 1767, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 942.04, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-05-25", "l_receiptdate": "1995-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ing to the slyly fin" }
+{ "l_orderkey": 2310, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 45217.92, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-20", "l_receiptdate": "1996-10-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ep slyly alongside of the " }
 { "l_orderkey": 2342, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11304.48, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-31", "l_commitdate": "1996-07-26", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "print blithely even deposits. carefull" }
+{ "l_orderkey": 2560, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8478.36, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-23", "l_commitdate": "1992-10-29", "l_receiptdate": "1992-11-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits affix quickly. unusual, eve" }
 { "l_orderkey": 2566, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2826.12, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-30", "l_receiptdate": "1992-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ckages are ironic Tiresias. furious" }
+{ "l_orderkey": 2599, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 24493.04, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-08", "l_commitdate": "1996-12-21", "l_receiptdate": "1996-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "nag carefully " }
 { "l_orderkey": 2659, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-01-24", "l_receiptdate": "1994-03-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "idle tithes" }
 { "l_orderkey": 2882, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 28261.2, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-15", "l_commitdate": "1995-10-13", "l_receiptdate": "1995-10-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "among the furiously even theodolites. regu" }
 { "l_orderkey": 2944, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 41449.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-11-10", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ickly. regular requests haggle. idea" }
+{ "l_orderkey": 3268, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37681.6, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-30", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly. bold, eve" }
+{ "l_orderkey": 3488, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11304.48, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-02-16", "l_receiptdate": "1995-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e slyly; furiously final packages wak" }
+{ "l_orderkey": 3585, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 42391.8, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-20", "l_commitdate": "1995-02-19", "l_receiptdate": "1995-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "are blithely c" }
+{ "l_orderkey": 4576, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13188.56, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-09-30", "l_receiptdate": "1996-09-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "detect slyly." }
+{ "l_orderkey": 4579, "l_partkey": 42, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26377.12, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-22", "l_commitdate": "1996-02-13", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly across the " }
+{ "l_orderkey": 4645, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 25435.08, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-10-25", "l_receiptdate": "1994-12-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ously express pinto beans. ironic depos" }
+{ "l_orderkey": 4837, "l_partkey": 42, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-12", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-08-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ing requests are blithely regular instructi" }
+{ "l_orderkey": 4994, "l_partkey": 42, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22608.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-09-24", "l_receiptdate": "1996-08-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s. slyly ironic deposits cajole f" }
+{ "l_orderkey": 5442, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 15072.64, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-12", "l_commitdate": "1998-03-03", "l_receiptdate": "1998-05-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages. accounts haggle dependencies. f" }
+{ "l_orderkey": 5765, "l_partkey": 42, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19782.84, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-05", "l_commitdate": "1995-02-12", "l_receiptdate": "1995-05-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ole furiously. quick, special dependencies " }
+{ "l_orderkey": 67, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21643.92, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly regular deposit" }
 { "l_orderkey": 98, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 26349.12, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-24", "l_commitdate": "1994-10-25", "l_receiptdate": "1995-01-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " pending, regular accounts s" }
 { "l_orderkey": 226, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-17", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully pending pi" }
-{ "l_orderkey": 3878, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18820.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the furiously careful ideas cajole slyly sl" }
-{ "l_orderkey": 1314, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10351.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tegrate furious" }
-{ "l_orderkey": 2085, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-01-11", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". carefully e" }
-{ "l_orderkey": 2852, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the blithe" }
-{ "l_orderkey": 3617, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20702.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily even accounts. packages sleep blithe" }
-{ "l_orderkey": 3941, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44228.88, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully pending" }
-{ "l_orderkey": 4165, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11292.48, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-20", "l_receiptdate": "1997-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nwind slow theodolites. carefully pending " }
-{ "l_orderkey": 4512, "l_partkey": 41, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-16", "l_receiptdate": "1995-12-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly regular pinto beans. carefully bold depo" }
-{ "l_orderkey": 67, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21643.92, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-02-14", "l_receiptdate": "1997-05-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly regular deposit" }
 { "l_orderkey": 517, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8469.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-03", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-05-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " slyly stealthily express instructions. " }
+{ "l_orderkey": 1314, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10351.44, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-16", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "tegrate furious" }
+{ "l_orderkey": 1316, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dugouts. co" }
+{ "l_orderkey": 2085, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-01-11", "l_receiptdate": "1994-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". carefully e" }
 { "l_orderkey": 2279, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35759.52, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-06", "l_receiptdate": "1993-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s above the furiously express dep" }
+{ "l_orderkey": 2406, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "gular accounts caj" }
+{ "l_orderkey": 2852, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " the blithe" }
 { "l_orderkey": 2917, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34818.48, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-12", "l_commitdate": "1998-02-03", "l_receiptdate": "1997-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dependencies. express " }
+{ "l_orderkey": 3073, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23526.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nag asymptotes. pinto beans sleep " }
 { "l_orderkey": 3170, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31995.36, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-01", "l_commitdate": "1998-01-11", "l_receiptdate": "1998-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s about the fluffily final de" }
+{ "l_orderkey": 3367, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25408.08, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-03-16", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kly even instructions caj" }
 { "l_orderkey": 3459, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42346.8, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-31", "l_commitdate": "1994-09-09", "l_receiptdate": "1994-08-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ntly speci" }
+{ "l_orderkey": 3617, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20702.88, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-07-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uffily even accounts. packages sleep blithe" }
+{ "l_orderkey": 3878, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18820.8, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-20", "l_commitdate": "1997-05-24", "l_receiptdate": "1997-07-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the furiously careful ideas cajole slyly sl" }
+{ "l_orderkey": 3941, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44228.88, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-09", "l_receiptdate": "1996-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " carefully pending" }
+{ "l_orderkey": 4001, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17879.76, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. carefully ironi" }
+{ "l_orderkey": 4165, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11292.48, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-20", "l_receiptdate": "1997-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "nwind slow theodolites. carefully pending " }
 { "l_orderkey": 4448, "l_partkey": 41, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32936.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-18", "l_commitdate": "1998-07-27", "l_receiptdate": "1998-10-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "aggle carefully alongside of the q" }
+{ "l_orderkey": 4512, "l_partkey": 41, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22584.96, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-16", "l_commitdate": "1996-01-16", "l_receiptdate": "1995-12-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly regular pinto beans. carefully bold depo" }
 { "l_orderkey": 4869, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 29172.24, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-17", "l_commitdate": "1994-11-30", "l_receiptdate": "1995-02-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ins. always unusual ideas across the ir" }
 { "l_orderkey": 4896, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 17879.76, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-13", "l_commitdate": "1992-11-13", "l_receiptdate": "1993-01-09", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "nusual requ" }
 { "l_orderkey": 4964, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 39523.68, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-04", "l_commitdate": "1997-08-28", "l_receiptdate": "1997-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " hinder. idly even" }
-{ "l_orderkey": 1316, "l_partkey": 41, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-04", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-02-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "l dugouts. co" }
-{ "l_orderkey": 2406, "l_partkey": 41, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37641.6, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-02", "l_receiptdate": "1997-01-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "gular accounts caj" }
-{ "l_orderkey": 3073, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23526.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-14", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-04-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nag asymptotes. pinto beans sleep " }
-{ "l_orderkey": 3367, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 25408.08, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-13", "l_commitdate": "1993-03-16", "l_receiptdate": "1993-04-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kly even instructions caj" }
-{ "l_orderkey": 4001, "l_partkey": 41, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17879.76, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-23", "l_commitdate": "1997-06-15", "l_receiptdate": "1997-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ackages. carefully ironi" }
 { "l_orderkey": 129, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 31021.32, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-08", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "sts nag bravely. fluffily" }
-{ "l_orderkey": 1667, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-11-24", "l_receiptdate": "1998-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "around the pinto beans. express, special" }
-{ "l_orderkey": 1924, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19740.84, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-21", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " blithely reg" }
-{ "l_orderkey": 1926, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27261.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hily unusual packages are fluffily am" }
-{ "l_orderkey": 4231, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32901.4, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "le quickly regular, unus" }
-{ "l_orderkey": 4802, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual accounts wake blithely. b" }
-{ "l_orderkey": 4935, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34781.48, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y even dependencies nag a" }
-{ "l_orderkey": 5505, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16920.72, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-25", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " to the quickly express pac" }
-{ "l_orderkey": 5829, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ithely; accounts cajole ideas. regular foxe" }
-{ "l_orderkey": 1252, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-10-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake carefully-- packages sleep. quick " }
-{ "l_orderkey": 1543, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8460.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ravely special requests " }
-{ "l_orderkey": 3366, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-25", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully about " }
-{ "l_orderkey": 417, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36661.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y regular requests wake along " }
-{ "l_orderkey": 3110, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15040.64, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-10", "l_commitdate": "1995-02-06", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the regular acco" }
-{ "l_orderkey": 3170, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11280.48, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-17", "l_receiptdate": "1998-02-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing accounts along the speci" }
-{ "l_orderkey": 3462, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40421.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-01", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully. final, final ideas sleep slyly" }
-{ "l_orderkey": 4064, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14100.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "braids affix across the regular sheave" }
-{ "l_orderkey": 5286, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y special a" }
-{ "l_orderkey": 5347, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-04-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ldly pending asymptotes ki" }
 { "l_orderkey": 289, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 45121.92, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-03-30", "l_receiptdate": "1997-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "sits cajole. bold pinto beans x-ray fl" }
+{ "l_orderkey": 417, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36661.56, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-31", "l_commitdate": "1994-05-02", "l_receiptdate": "1994-06-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y regular requests wake along " }
 { "l_orderkey": 773, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 40421.72, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-06", "l_commitdate": "1993-11-20", "l_receiptdate": "1993-11-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "furiously bold dependencies. blithel" }
 { "l_orderkey": 931, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16920.72, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-04", "l_commitdate": "1993-01-11", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "slyly ironic re" }
 { "l_orderkey": 993, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9400.4, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-11-13", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "encies wake fur" }
+{ "l_orderkey": 1252, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-23", "l_receiptdate": "1997-10-18", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake carefully-- packages sleep. quick " }
 { "l_orderkey": 1507, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 31021.32, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-23", "l_receiptdate": "1993-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " asymptotes nag furiously above t" }
+{ "l_orderkey": 1543, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8460.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-14", "l_commitdate": "1997-05-19", "l_receiptdate": "1997-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ravely special requests " }
+{ "l_orderkey": 1667, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17860.76, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-23", "l_commitdate": "1997-11-24", "l_receiptdate": "1998-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "around the pinto beans. express, special" }
+{ "l_orderkey": 1924, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 19740.84, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-21", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " blithely reg" }
+{ "l_orderkey": 1926, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27261.16, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "hily unusual packages are fluffily am" }
 { "l_orderkey": 2818, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 30081.28, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-02-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "arefully! ac" }
+{ "l_orderkey": 3110, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 15040.64, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-10", "l_commitdate": "1995-02-06", "l_receiptdate": "1995-01-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "across the regular acco" }
 { "l_orderkey": 3139, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 43241.84, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-28", "l_commitdate": "1992-03-04", "l_receiptdate": "1992-05-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "of the unusual, unusual re" }
+{ "l_orderkey": 3170, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11280.48, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-12", "l_commitdate": "1998-01-17", "l_receiptdate": "1998-02-24", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ing accounts along the speci" }
+{ "l_orderkey": 3366, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-25", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " carefully about " }
+{ "l_orderkey": 3462, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40421.72, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-01", "l_commitdate": "1997-07-18", "l_receiptdate": "1997-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " carefully. final, final ideas sleep slyly" }
 { "l_orderkey": 3906, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 47002.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-08-24", "l_receiptdate": "1992-09-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ke slyly. stealt" }
 { "l_orderkey": 3973, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37601.6, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-03", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "g the carefully blithe f" }
 { "l_orderkey": 4002, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-07-07", "l_receiptdate": "1997-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " furiously furiously special theodoli" }
+{ "l_orderkey": 4064, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14100.6, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-09", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-11-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "braids affix across the regular sheave" }
+{ "l_orderkey": 4231, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32901.4, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-21", "l_commitdate": "1998-01-24", "l_receiptdate": "1998-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "le quickly regular, unus" }
 { "l_orderkey": 4292, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 940.04, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-07", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " the furiously ev" }
 { "l_orderkey": 4646, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 35721.52, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-23", "l_receiptdate": "1996-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "al platelets cajole. slyly final dol" }
+{ "l_orderkey": 4802, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-16", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-04-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "unusual accounts wake blithely. b" }
+{ "l_orderkey": 4935, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34781.48, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-30", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y even dependencies nag a" }
 { "l_orderkey": 4995, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8460.36, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " ironic packages cajole across t" }
 { "l_orderkey": 4997, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4700.2, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-04-24", "l_receiptdate": "1998-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "aggle slyly alongside of the slyly i" }
+{ "l_orderkey": 5286, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5640.24, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-12-05", "l_receiptdate": "1997-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "y special a" }
+{ "l_orderkey": 5347, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-04-03", "l_receiptdate": "1995-04-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ldly pending asymptotes ki" }
+{ "l_orderkey": 5505, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16920.72, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-25", "l_commitdate": "1997-12-12", "l_receiptdate": "1997-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " to the quickly express pac" }
+{ "l_orderkey": 5829, "l_partkey": 40, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3760.16, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-01", "l_commitdate": "1997-02-17", "l_receiptdate": "1997-03-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ithely; accounts cajole ideas. regular foxe" }
 { "l_orderkey": 5959, "l_partkey": 40, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34781.48, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-05", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-06-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "endencies. brai" }
 { "l_orderkey": 100, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13146.42, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y. furiously ironic ideas gr" }
-{ "l_orderkey": 896, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44134.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even pinto beans integrate. b" }
-{ "l_orderkey": 2435, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7512.24, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e fluffily quickly final accounts. care" }
 { "l_orderkey": 482, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 43195.38, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-19", "l_commitdate": "1996-06-05", "l_receiptdate": "1996-08-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "furiously thin realms. final, fina" }
-{ "l_orderkey": 1569, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15024.48, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-26", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "deposits. blithely final asymptotes ac" }
-{ "l_orderkey": 2535, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11268.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uses sleep among the packages. excuses " }
-{ "l_orderkey": 4994, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 37561.2, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits. regula" }
+{ "l_orderkey": 612, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26292.84, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-05", "l_receiptdate": "1992-12-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly regular asym" }
+{ "l_orderkey": 896, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 44134.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-06-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ly even pinto beans integrate. b" }
 { "l_orderkey": 1447, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8451.27, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-15", "l_commitdate": "1993-01-07", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "counts wake s" }
+{ "l_orderkey": 1569, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 15024.48, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-26", "l_commitdate": "1998-06-16", "l_receiptdate": "1998-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "deposits. blithely final asymptotes ac" }
 { "l_orderkey": 1600, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7512.24, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-07", "l_commitdate": "1993-04-22", "l_receiptdate": "1993-03-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cajole furiously fluf" }
 { "l_orderkey": 1603, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 939.03, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-17", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "d accounts. special warthogs use fur" }
 { "l_orderkey": 1958, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 29.0, "l_extendedprice": 27231.87, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-14", "l_commitdate": "1995-11-06", "l_receiptdate": "1995-11-01", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "final requests nag according to the " }
+{ "l_orderkey": 2435, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7512.24, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-08", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-06-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e fluffily quickly final accounts. care" }
+{ "l_orderkey": 2535, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11268.36, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-07-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uses sleep among the packages. excuses " }
 { "l_orderkey": 2596, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17841.57, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-02", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-09-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ias mold! sp" }
-{ "l_orderkey": 4774, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3756.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "xes according to the foxes wake above the f" }
-{ "l_orderkey": 5287, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30048.96, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-01-27", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "heodolites haggle caref" }
-{ "l_orderkey": 5344, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19719.63, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-09-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "xes. furiously even pinto beans sleep f" }
-{ "l_orderkey": 5570, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14085.45, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "beans nag slyly special, regular pack" }
-{ "l_orderkey": 5958, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16902.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "regular requests haggle" }
-{ "l_orderkey": 612, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26292.84, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-12", "l_commitdate": "1992-12-05", "l_receiptdate": "1992-12-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly regular asym" }
 { "l_orderkey": 3396, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 16902.54, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-08-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "l requests haggle furiously along the fur" }
 { "l_orderkey": 3461, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 41317.32, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-03", "l_receiptdate": "1993-05-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " haggle quickly even ideas. fin" }
 { "l_orderkey": 3588, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 43195.38, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-06-06", "l_commitdate": "1995-05-08", "l_receiptdate": "1995-06-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " slyly ironic deposits sublate ab" }
 { "l_orderkey": 4515, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 14085.45, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-26", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-06-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "posits wake" }
+{ "l_orderkey": 4774, "l_partkey": 39, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3756.12, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-05-30", "l_receiptdate": "1993-08-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "xes according to the foxes wake above the f" }
+{ "l_orderkey": 4994, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 37561.2, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-25", "l_commitdate": "1996-08-16", "l_receiptdate": "1996-09-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "eposits. regula" }
+{ "l_orderkey": 5287, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 30048.96, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-01-27", "l_receiptdate": "1994-02-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "heodolites haggle caref" }
+{ "l_orderkey": 5344, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19719.63, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-09-02", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "xes. furiously even pinto beans sleep f" }
+{ "l_orderkey": 5570, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 14085.45, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "beans nag slyly special, regular pack" }
+{ "l_orderkey": 5958, "l_partkey": 39, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16902.54, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-02", "l_commitdate": "1995-10-17", "l_receiptdate": "1995-12-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "regular requests haggle" }
+{ "l_orderkey": 5, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-10-13", "l_receiptdate": "1994-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites. fluffily unusual" }
+{ "l_orderkey": 69, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2814.09, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-07-27", "l_receiptdate": "1994-06-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithely final d" }
 { "l_orderkey": 70, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 34707.11, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-03-16", "l_receiptdate": "1994-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "n accounts are. q" }
-{ "l_orderkey": 1026, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33769.08, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-14", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st the ide" }
-{ "l_orderkey": 1699, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "to the final requests are carefully silent " }
-{ "l_orderkey": 2213, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40335.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages are along the carefully bol" }
-{ "l_orderkey": 3462, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13132.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "yly. blithely bold theodolites wa" }
-{ "l_orderkey": 4613, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15946.51, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "liers cajole a" }
-{ "l_orderkey": 4864, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35645.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-07", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ording to the ironic, ir" }
+{ "l_orderkey": 226, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32831.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "osits cajole. final, even foxes a" }
+{ "l_orderkey": 322, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4690.15, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-15", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special grouches sleep quickly instructio" }
 { "l_orderkey": 804, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19698.63, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-12", "l_commitdate": "1993-06-06", "l_receiptdate": "1993-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ular, ironic foxes. quickly even accounts" }
+{ "l_orderkey": 992, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s use silently. blithely regular ideas b" }
+{ "l_orderkey": 1026, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33769.08, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-14", "l_commitdate": "1997-07-20", "l_receiptdate": "1997-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st the ide" }
 { "l_orderkey": 1091, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 37521.2, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "platelets. regular packag" }
 { "l_orderkey": 1414, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36583.17, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-22", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-10-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "quickly aro" }
+{ "l_orderkey": 1699, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "to the final requests are carefully silent " }
+{ "l_orderkey": 2023, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1876.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-27", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing packages. fluffily silen" }
+{ "l_orderkey": 2213, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 40335.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-18", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-05-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r packages are along the carefully bol" }
 { "l_orderkey": 3014, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28140.9, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " final foxes." }
 { "l_orderkey": 3104, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24388.78, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1993-12-05", "l_receiptdate": "1994-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "es boost carefully. slyly " }
 { "l_orderkey": 3266, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40335.29, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-04", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-05-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular asymptotes use careful" }
-{ "l_orderkey": 4033, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "t the blithely dogg" }
-{ "l_orderkey": 5798, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8442.27, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e blithely" }
-{ "l_orderkey": 226, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32831.05, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-31", "l_commitdate": "1993-05-18", "l_receiptdate": "1993-04-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "osits cajole. final, even foxes a" }
-{ "l_orderkey": 2023, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1876.06, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-27", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ing packages. fluffily silen" }
+{ "l_orderkey": 3269, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43149.38, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "final asymptotes nag" }
 { "l_orderkey": 3270, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 41273.32, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-20", "l_commitdate": "1997-08-15", "l_receiptdate": "1997-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " accounts. carefully even " }
+{ "l_orderkey": 3462, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13132.42, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-31", "l_commitdate": "1997-07-05", "l_receiptdate": "1997-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "yly. blithely bold theodolites wa" }
 { "l_orderkey": 3904, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20636.66, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-02-09", "l_receiptdate": "1998-02-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "structions cajole carefully. carefully f" }
+{ "l_orderkey": 4033, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-19", "l_commitdate": "1993-08-05", "l_receiptdate": "1993-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "t the blithely dogg" }
+{ "l_orderkey": 4613, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15946.51, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-07", "l_commitdate": "1998-05-11", "l_receiptdate": "1998-06-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "liers cajole a" }
+{ "l_orderkey": 4864, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35645.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-07", "l_receiptdate": "1993-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ording to the ironic, ir" }
 { "l_orderkey": 4993, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-21", "l_commitdate": "1994-10-31", "l_receiptdate": "1994-09-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ular, pending packages at the even packa" }
 { "l_orderkey": 5601, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 27202.87, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-02-24", "l_receiptdate": "1992-04-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " ironic ideas. final" }
-{ "l_orderkey": 5, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 46901.5, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-10-13", "l_receiptdate": "1994-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites. fluffily unusual" }
-{ "l_orderkey": 69, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2814.09, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-06", "l_commitdate": "1994-07-27", "l_receiptdate": "1994-06-15", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " blithely final d" }
-{ "l_orderkey": 322, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4690.15, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-15", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-04-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " special grouches sleep quickly instructio" }
-{ "l_orderkey": 992, "l_partkey": 38, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31893.02, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-29", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-11-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s use silently. blithely regular ideas b" }
-{ "l_orderkey": 3269, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 43149.38, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-05-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "final asymptotes nag" }
-{ "l_orderkey": 1319, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11244.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "packages integrate furiously. expres" }
-{ "l_orderkey": 1923, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8433.27, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-29", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lites. ironic instructions integrate bravel" }
-{ "l_orderkey": 2980, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1874.06, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "enly across the special, pending packag" }
-{ "l_orderkey": 3074, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46851.5, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending requests haggle s" }
-{ "l_orderkey": 4224, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18740.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-09", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts promise across the requests. blith" }
+{ "l_orderkey": 5798, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 9.0, "l_extendedprice": 8442.27, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-05-25", "l_receiptdate": "1998-05-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "e blithely" }
+{ "l_orderkey": 163, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25299.81, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ously express dependen" }
+{ "l_orderkey": 576, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5622.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. ironic multipliers " }
+{ "l_orderkey": 678, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20614.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-10", "l_commitdate": "1993-04-29", "l_receiptdate": "1993-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "de of the carefully even requests. bl" }
 { "l_orderkey": 1026, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5622.18, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-07", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "to beans. special, regular packages hagg" }
 { "l_orderkey": 1088, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10307.33, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-30", "l_commitdate": "1992-07-25", "l_receiptdate": "1992-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "inal requests. fluffily express theod" }
 { "l_orderkey": 1220, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2811.09, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-06", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final theodolites. blithely silent " }
-{ "l_orderkey": 2500, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31859.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " stealthy a" }
-{ "l_orderkey": 163, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 27.0, "l_extendedprice": 25299.81, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-26", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ously express dependen" }
+{ "l_orderkey": 1319, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11244.36, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "packages integrate furiously. expres" }
+{ "l_orderkey": 1923, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8433.27, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-29", "l_commitdate": "1997-09-13", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "lites. ironic instructions integrate bravel" }
 { "l_orderkey": 1923, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 46851.5, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-04", "l_commitdate": "1997-08-08", "l_receiptdate": "1997-11-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "uickly along the bold courts. bold the" }
+{ "l_orderkey": 2500, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31859.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": " stealthy a" }
+{ "l_orderkey": 2980, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 1874.06, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-10-22", "l_receiptdate": "1996-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "enly across the special, pending packag" }
 { "l_orderkey": 2981, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 13118.42, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-30", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "kages detect furiously express requests." }
+{ "l_orderkey": 3074, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46851.5, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-31", "l_commitdate": "1992-12-15", "l_receiptdate": "1993-02-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously pending requests haggle s" }
 { "l_orderkey": 3589, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39355.26, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-11", "l_commitdate": "1994-07-17", "l_receiptdate": "1994-08-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "he blithely unusual pac" }
+{ "l_orderkey": 4224, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18740.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-09", "l_commitdate": "1997-08-23", "l_receiptdate": "1997-11-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts promise across the requests. blith" }
 { "l_orderkey": 4645, "l_partkey": 37, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 42.0, "l_extendedprice": 39355.26, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-02", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "regular pinto beans amon" }
 { "l_orderkey": 5057, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 35607.14, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-24", "l_commitdate": "1997-09-07", "l_receiptdate": "1997-10-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "packages. stealthily bold wa" }
-{ "l_orderkey": 576, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5622.18, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-28", "l_commitdate": "1997-06-16", "l_receiptdate": "1997-09-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ts. ironic multipliers " }
-{ "l_orderkey": 678, "l_partkey": 37, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20614.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-10", "l_commitdate": "1993-04-29", "l_receiptdate": "1993-06-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "de of the carefully even requests. bl" }
-{ "l_orderkey": 962, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25272.81, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-11", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y slyly express deposits. final i" }
-{ "l_orderkey": 1154, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16848.54, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-26", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular excuses cajole blithely. fi" }
-{ "l_orderkey": 2404, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "from the final orbits? even pinto beans hag" }
-{ "l_orderkey": 3296, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1994-12-23", "l_receiptdate": "1995-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "carefully fur" }
-{ "l_orderkey": 3302, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42121.35, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "counts use quickl" }
-{ "l_orderkey": 3303, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24336.78, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly permanent requests w" }
-{ "l_orderkey": 3521, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26208.84, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly above the slyly final" }
 { "l_orderkey": 134, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11232.36, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-07-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "nts are quic" }
-{ "l_orderkey": 2371, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 29952.96, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-02-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ruthless accounts. " }
-{ "l_orderkey": 2853, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully slyly quick packages. final c" }
-{ "l_orderkey": 1575, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36505.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ironic requests snooze ironic, regular acc" }
-{ "l_orderkey": 4038, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-09", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " special instructions. packa" }
-{ "l_orderkey": 4484, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27144.87, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " wake blithely ironic" }
-{ "l_orderkey": 4768, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4680.15, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "egular accounts. bravely final fra" }
-{ "l_orderkey": 4833, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17784.57, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-07-09", "l_receiptdate": "1996-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y quick theodolit" }
 { "l_orderkey": 258, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 23400.75, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-13", "l_commitdate": "1994-02-26", "l_receiptdate": "1994-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "leep pending packages." }
+{ "l_orderkey": 962, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25272.81, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-11", "l_commitdate": "1994-07-10", "l_receiptdate": "1994-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y slyly express deposits. final i" }
 { "l_orderkey": 1126, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41185.32, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-07", "l_commitdate": "1998-04-02", "l_receiptdate": "1998-05-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "es. carefully special" }
+{ "l_orderkey": 1154, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16848.54, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-26", "l_commitdate": "1992-03-24", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "y regular excuses cajole blithely. fi" }
+{ "l_orderkey": 1575, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36505.17, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-10-15", "l_receiptdate": "1995-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ironic requests snooze ironic, regular acc" }
 { "l_orderkey": 1924, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 17.0, "l_extendedprice": 15912.51, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-31", "l_commitdate": "1996-11-12", "l_receiptdate": "1997-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "e carefully theodolites. ironically ironic " }
 { "l_orderkey": 1988, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25272.81, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-27", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "uests. regular requests are according to t" }
 { "l_orderkey": 2342, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-31", "l_commitdate": "1996-08-09", "l_receiptdate": "1996-09-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ffily. unusual pinto beans wake c" }
+{ "l_orderkey": 2371, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 29952.96, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-02-23", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the ruthless accounts. " }
+{ "l_orderkey": 2404, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-22", "l_commitdate": "1997-06-06", "l_receiptdate": "1997-05-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "from the final orbits? even pinto beans hag" }
+{ "l_orderkey": 2853, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 936.03, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-01", "l_commitdate": "1994-06-27", "l_receiptdate": "1994-09-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "refully slyly quick packages. final c" }
+{ "l_orderkey": 3296, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-03", "l_commitdate": "1994-12-23", "l_receiptdate": "1995-01-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "carefully fur" }
+{ "l_orderkey": 3302, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 42121.35, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "counts use quickl" }
+{ "l_orderkey": 3303, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24336.78, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-18", "l_commitdate": "1998-03-11", "l_receiptdate": "1998-02-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ickly permanent requests w" }
 { "l_orderkey": 3395, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35569.14, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1995-01-13", "l_receiptdate": "1995-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " silent accounts are blithely" }
+{ "l_orderkey": 3521, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 26208.84, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-22", "l_receiptdate": "1993-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "e slyly above the slyly final" }
+{ "l_orderkey": 4038, "l_partkey": 36, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5616.18, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-09", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " special instructions. packa" }
 { "l_orderkey": 4131, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7488.24, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-03", "l_commitdate": "1998-03-15", "l_receiptdate": "1998-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " after the furiously ironic d" }
+{ "l_orderkey": 4484, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 29.0, "l_extendedprice": 27144.87, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-27", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " wake blithely ironic" }
 { "l_orderkey": 4641, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 14040.45, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-25", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s. carefully even exc" }
+{ "l_orderkey": 4768, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4680.15, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-27", "l_commitdate": "1994-02-09", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "egular accounts. bravely final fra" }
+{ "l_orderkey": 4833, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17784.57, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-07-09", "l_receiptdate": "1996-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y quick theodolit" }
 { "l_orderkey": 4901, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 38377.23, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-18", "l_commitdate": "1998-02-18", "l_receiptdate": "1998-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "efully bold packages affix carefully eve" }
 { "l_orderkey": 5666, "l_partkey": 36, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13104.42, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lar deposits nag against the slyly final d" }
+{ "l_orderkey": 68, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43011.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "egular dependencies affix ironically along " }
 { "l_orderkey": 71, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 42076.35, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-23", "l_commitdate": "1998-03-20", "l_receiptdate": "1998-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " ironic packages believe blithely a" }
 { "l_orderkey": 326, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-10-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cies sleep quick" }
-{ "l_orderkey": 2049, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-25", "l_receiptdate": "1995-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages are slyly alongside" }
-{ "l_orderkey": 2880, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-26", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "even requests. quick" }
-{ "l_orderkey": 4418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29920.96, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-02", "l_receiptdate": "1993-05-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. bold pinto b" }
-{ "l_orderkey": 4769, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14960.48, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " deposits. slyly even asymptote" }
-{ "l_orderkey": 5315, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11220.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts. furiously ironi" }
-{ "l_orderkey": 5414, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21505.69, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-08-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e bold, express dolphins. spec" }
-{ "l_orderkey": 2724, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 935.03, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1994-11-27", "l_receiptdate": "1995-01-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly carefully blithe theodolites-- pl" }
-{ "l_orderkey": 2914, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-06-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s integrate. bold deposits sleep req" }
-{ "l_orderkey": 3460, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "o the even deposits" }
-{ "l_orderkey": 4230, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28050.9, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s. final excuses across the" }
-{ "l_orderkey": 5153, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39271.26, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re thinly. ironic" }
-{ "l_orderkey": 68, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 43011.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-07-08", "l_receiptdate": "1998-08-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "egular dependencies affix ironically along " }
 { "l_orderkey": 418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2805.09, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-29", "l_commitdate": "1995-07-12", "l_receiptdate": "1995-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly furiously regular w" }
 { "l_orderkey": 1697, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17765.57, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-08", "l_commitdate": "1996-11-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ons? special, special accounts after" }
 { "l_orderkey": 1701, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24310.78, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-07-11", "l_receiptdate": "1992-07-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " accounts. blithely pending pinto be" }
 { "l_orderkey": 2048, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6545.21, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-07", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lent platelets boost deposits. carefully sp" }
-{ "l_orderkey": 4611, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " final pinto beans. permanent, sp" }
-{ "l_orderkey": 4804, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38336.23, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". deposits haggle express tithes?" }
-{ "l_orderkey": 4833, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-16", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y pending packages sleep blithely regular r" }
+{ "l_orderkey": 2049, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-25", "l_commitdate": "1996-02-25", "l_receiptdate": "1995-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " packages are slyly alongside" }
 { "l_orderkey": 2371, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19635.63, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-02-14", "l_receiptdate": "1998-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gle furiously regu" }
+{ "l_orderkey": 2724, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 935.03, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-26", "l_commitdate": "1994-11-27", "l_receiptdate": "1995-01-07", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly carefully blithe theodolites-- pl" }
+{ "l_orderkey": 2880, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-26", "l_commitdate": "1992-06-01", "l_receiptdate": "1992-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "even requests. quick" }
+{ "l_orderkey": 2914, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-11", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-06-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s integrate. bold deposits sleep req" }
 { "l_orderkey": 3270, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10285.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-29", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-08-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " solve at the regular deposits. " }
+{ "l_orderkey": 3460, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37401.2, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-29", "l_commitdate": "1995-11-10", "l_receiptdate": "1995-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "o the even deposits" }
+{ "l_orderkey": 4230, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 28050.9, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-11", "l_commitdate": "1992-04-29", "l_receiptdate": "1992-03-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s. final excuses across the" }
 { "l_orderkey": 4257, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4675.15, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-06-05", "l_receiptdate": "1995-05-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "n deposits. furiously e" }
 { "l_orderkey": 4258, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20570.66, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-12-06", "l_receiptdate": "1996-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "e regular, even asym" }
+{ "l_orderkey": 4418, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29920.96, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-02", "l_receiptdate": "1993-05-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ly. bold pinto b" }
+{ "l_orderkey": 4611, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28985.93, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-02-14", "l_receiptdate": "1993-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " final pinto beans. permanent, sp" }
+{ "l_orderkey": 4769, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14960.48, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-16", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " deposits. slyly even asymptote" }
+{ "l_orderkey": 4804, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 38336.23, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-06", "l_commitdate": "1992-04-12", "l_receiptdate": "1992-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": ". deposits haggle express tithes?" }
+{ "l_orderkey": 4833, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 3740.12, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-16", "l_commitdate": "1996-06-29", "l_receiptdate": "1996-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y pending packages sleep blithely regular r" }
+{ "l_orderkey": 5153, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39271.26, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-03", "l_commitdate": "1995-11-09", "l_receiptdate": "1995-10-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "re thinly. ironic" }
+{ "l_orderkey": 5315, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11220.36, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-18", "l_commitdate": "1993-01-16", "l_receiptdate": "1993-01-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts. furiously ironi" }
+{ "l_orderkey": 5414, "l_partkey": 35, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21505.69, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-22", "l_commitdate": "1993-05-26", "l_receiptdate": "1993-08-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "e bold, express dolphins. spec" }
 { "l_orderkey": 5856, "l_partkey": 35, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 32726.05, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1994-12-23", "l_receiptdate": "1994-11-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "excuses. finally ir" }
-{ "l_orderkey": 645, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-03-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep. slyly even " }
-{ "l_orderkey": 1381, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously regular package" }
-{ "l_orderkey": 2272, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37361.2, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely ir" }
-{ "l_orderkey": 2722, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14944.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts besides the fluffy," }
-{ "l_orderkey": 3270, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "promise carefully." }
-{ "l_orderkey": 4743, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25218.81, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-26", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aids use. express deposits" }
-{ "l_orderkey": 576, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5604.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. slyly even sauternes a" }
-{ "l_orderkey": 1284, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8406.27, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "after the pending" }
-{ "l_orderkey": 1443, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43899.41, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-05", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "carefully ironic requests sl" }
-{ "l_orderkey": 1478, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19614.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " fluffily pending acc" }
-{ "l_orderkey": 1920, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-22", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ickly ironic d" }
-{ "l_orderkey": 2307, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ven instructions wake fluffily " }
-{ "l_orderkey": 4646, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans sleep car" }
-{ "l_orderkey": 4899, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1994-01-10", "l_receiptdate": "1993-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " foxes eat" }
-{ "l_orderkey": 5345, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "leep slyly regular fox" }
 { "l_orderkey": 33, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 38295.23, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1994-01-24", "l_receiptdate": "1993-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unusual packages doubt caref" }
-{ "l_orderkey": 961, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27086.87, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l accounts use blithely against the" }
-{ "l_orderkey": 1571, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 22416.72, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "warthogs wake carefully acro" }
-{ "l_orderkey": 1733, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gainst the final deposits. carefully final " }
-{ "l_orderkey": 1766, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites above the final, regular acc" }
-{ "l_orderkey": 2275, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28020.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re slyly slyly special idea" }
-{ "l_orderkey": 2497, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26152.84, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-21", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ouches. special, regular requests" }
-{ "l_orderkey": 3845, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41097.32, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s haggle among the fluffily regula" }
-{ "l_orderkey": 4614, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-09-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ickly furio" }
-{ "l_orderkey": 4707, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial sheaves boost blithely accor" }
 { "l_orderkey": 165, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2802.09, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-29", "l_commitdate": "1993-03-06", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "riously requests. depos" }
 { "l_orderkey": 230, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7472.24, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-21", "l_commitdate": "1994-01-05", "l_receiptdate": "1993-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "nal ideas. silent, reg" }
 { "l_orderkey": 322, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2802.09, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-03", "l_commitdate": "1992-05-10", "l_receiptdate": "1992-07-28", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ending, ironic deposits along the blith" }
+{ "l_orderkey": 576, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5604.18, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-07-26", "l_receiptdate": "1997-06-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "al deposits. slyly even sauternes a" }
+{ "l_orderkey": 645, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-02-08", "l_receiptdate": "1995-03-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ep. slyly even " }
+{ "l_orderkey": 961, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 27086.87, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-10", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-06-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "l accounts use blithely against the" }
+{ "l_orderkey": 1284, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8406.27, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-03", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-04-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "after the pending" }
+{ "l_orderkey": 1381, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-13", "l_commitdate": "1998-08-12", "l_receiptdate": "1998-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " furiously regular package" }
+{ "l_orderkey": 1443, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43899.41, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-05", "l_commitdate": "1997-02-02", "l_receiptdate": "1997-03-03", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "carefully ironic requests sl" }
+{ "l_orderkey": 1478, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19614.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-25", "l_receiptdate": "1997-10-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " fluffily pending acc" }
+{ "l_orderkey": 1571, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 24.0, "l_extendedprice": 22416.72, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-01-31", "l_receiptdate": "1993-04-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "warthogs wake carefully acro" }
+{ "l_orderkey": 1733, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-16", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "gainst the final deposits. carefully final " }
+{ "l_orderkey": 1766, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11208.36, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-18", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "heodolites above the final, regular acc" }
+{ "l_orderkey": 1920, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-22", "l_commitdate": "1998-08-10", "l_receiptdate": "1998-10-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ickly ironic d" }
 { "l_orderkey": 1924, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28954.93, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-10-19", "l_receiptdate": "1996-10-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " the slyly regular foxes. ruthle" }
 { "l_orderkey": 2020, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46701.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-12", "l_commitdate": "1993-08-28", "l_receiptdate": "1993-08-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ts against the pending ideas serve along" }
+{ "l_orderkey": 2272, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37361.2, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-25", "l_commitdate": "1993-07-12", "l_receiptdate": "1993-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "lithely ir" }
+{ "l_orderkey": 2275, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 28020.9, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-11-21", "l_receiptdate": "1993-01-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re slyly slyly special idea" }
+{ "l_orderkey": 2307, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ven instructions wake fluffily " }
+{ "l_orderkey": 2497, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26152.84, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-21", "l_receiptdate": "1992-12-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ouches. special, regular requests" }
+{ "l_orderkey": 2722, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14944.48, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-06-09", "l_receiptdate": "1994-05-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts besides the fluffy," }
+{ "l_orderkey": 3270, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-23", "l_commitdate": "1997-08-17", "l_receiptdate": "1997-09-27", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "promise carefully." }
+{ "l_orderkey": 3845, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 41097.32, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-07-15", "l_receiptdate": "1992-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s haggle among the fluffily regula" }
+{ "l_orderkey": 4614, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29888.96, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-21", "l_commitdate": "1996-05-28", "l_receiptdate": "1996-09-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ickly furio" }
+{ "l_orderkey": 4646, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16812.54, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-08-10", "l_receiptdate": "1996-07-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans sleep car" }
 { "l_orderkey": 4675, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24284.78, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-12-29", "l_receiptdate": "1993-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nts. express requests are quickly " }
+{ "l_orderkey": 4707, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6538.21, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-14", "l_commitdate": "1995-04-06", "l_receiptdate": "1995-06-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial sheaves boost blithely accor" }
+{ "l_orderkey": 4743, "l_partkey": 34, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 25218.81, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-26", "l_commitdate": "1993-05-27", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "aids use. express deposits" }
+{ "l_orderkey": 4899, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 13076.42, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-10", "l_commitdate": "1994-01-10", "l_receiptdate": "1993-11-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " foxes eat" }
 { "l_orderkey": 5089, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 38.0, "l_extendedprice": 35493.14, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-09-11", "l_receiptdate": "1992-12-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular instructions are" }
 { "l_orderkey": 5285, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22416.72, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-19", "l_commitdate": "1994-04-03", "l_receiptdate": "1994-04-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ess packages. quick, even deposits snooze b" }
-{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2799.09, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly enticing requ" }
-{ "l_orderkey": 1444, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1995-02-18", "l_receiptdate": "1994-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ss requests. ironic ideas wake above" }
-{ "l_orderkey": 2787, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3732.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-26", "l_commitdate": "1995-11-26", "l_receiptdate": "1996-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. instructions nag furiously according " }
-{ "l_orderkey": 3555, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27057.87, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. carefully s" }
-{ "l_orderkey": 4359, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20526.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-28", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts wake ironic deposits. ironic" }
-{ "l_orderkey": 5574, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully final dugouts. express foxes nag " }
-{ "l_orderkey": 5733, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36388.17, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "side of the" }
+{ "l_orderkey": 5345, "l_partkey": 34, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20548.66, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-27", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-09-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "leep slyly regular fox" }
+{ "l_orderkey": 388, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-02-26", "l_receiptdate": "1993-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "accounts sleep furiously" }
+{ "l_orderkey": 451, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "express excuses. blithely ironic pin" }
 { "l_orderkey": 483, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7464.24, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-22", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "osits. carefully fin" }
 { "l_orderkey": 512, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11196.36, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-21", "l_commitdate": "1995-08-03", "l_receiptdate": "1995-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "old furiously express deposits. specia" }
-{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17727.57, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-20", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-07-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ught to are bold foxes" }
-{ "l_orderkey": 388, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-21", "l_commitdate": "1993-02-26", "l_receiptdate": "1993-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "accounts sleep furiously" }
-{ "l_orderkey": 2053, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31723.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ions. unusual dependencies" }
-{ "l_orderkey": 2855, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46651.5, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans. deposits " }
-{ "l_orderkey": 3747, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely bold orbits mold furiously blit" }
-{ "l_orderkey": 5346, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5598.18, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "escapades sleep furiously beside the " }
-{ "l_orderkey": 451, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-01", "l_commitdate": "1998-08-05", "l_receiptdate": "1998-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "express excuses. blithely ironic pin" }
 { "l_orderkey": 579, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36388.17, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-21", "l_commitdate": "1998-06-03", "l_receiptdate": "1998-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ncies. furiously final r" }
 { "l_orderkey": 1156, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1997-01-14", "l_receiptdate": "1996-12-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "dolphins. fluffily ironic packages sleep re" }
 { "l_orderkey": 1280, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12129.39, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-06", "l_commitdate": "1993-03-11", "l_receiptdate": "1993-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "blithely final accounts use evenly " }
+{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2799.09, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "yly enticing requ" }
+{ "l_orderkey": 1377, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17727.57, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-20", "l_commitdate": "1998-06-27", "l_receiptdate": "1998-07-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ught to are bold foxes" }
+{ "l_orderkey": 1444, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 39187.26, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-16", "l_commitdate": "1995-02-18", "l_receiptdate": "1994-12-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ss requests. ironic ideas wake above" }
+{ "l_orderkey": 2053, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31723.02, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-03-20", "l_receiptdate": "1995-04-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ions. unusual dependencies" }
 { "l_orderkey": 2215, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27990.9, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-09-10", "l_receiptdate": "1996-08-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ckages caj" }
 { "l_orderkey": 2784, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41986.35, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-07", "l_receiptdate": "1998-02-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "yly along the asymptotes. reque" }
+{ "l_orderkey": 2787, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3732.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-26", "l_commitdate": "1995-11-26", "l_receiptdate": "1996-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. instructions nag furiously according " }
+{ "l_orderkey": 2855, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46651.5, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "beans. deposits " }
 { "l_orderkey": 3526, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18660.6, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-16", "l_commitdate": "1995-04-26", "l_receiptdate": "1995-06-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "kages. bold, special requests detect sl" }
+{ "l_orderkey": 3555, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 27057.87, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-02", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "deas. carefully s" }
+{ "l_orderkey": 3747, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-18", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-11-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ithely bold orbits mold furiously blit" }
 { "l_orderkey": 4163, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12129.39, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-17", "l_commitdate": "1993-03-13", "l_receiptdate": "1993-03-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "phins wake. pending requests inte" }
 { "l_orderkey": 4227, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7464.24, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-04-30", "l_receiptdate": "1995-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " packages since the bold, u" }
+{ "l_orderkey": 4359, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20526.66, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-28", "l_commitdate": "1993-06-01", "l_receiptdate": "1993-04-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "accounts wake ironic deposits. ironic" }
+{ "l_orderkey": 5346, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5598.18, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-01", "l_commitdate": "1994-02-04", "l_receiptdate": "1994-03-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "escapades sleep furiously beside the " }
 { "l_orderkey": 5351, "l_partkey": 33, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43852.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-08-08", "l_receiptdate": "1998-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. grouches cajole. sile" }
+{ "l_orderkey": 5574, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19593.63, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-22", "l_commitdate": "1992-04-26", "l_receiptdate": "1992-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "fully final dugouts. express foxes nag " }
+{ "l_orderkey": 5733, "l_partkey": 33, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36388.17, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-22", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "side of the" }
+{ "l_orderkey": 129, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uests. foxes cajole slyly after the ca" }
 { "l_orderkey": 484, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 41941.35, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-03-20", "l_receiptdate": "1997-04-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly final excuses boost slyly blithe" }
+{ "l_orderkey": 640, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 41941.35, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ong the qui" }
+{ "l_orderkey": 775, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14912.48, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "un quickly slyly" }
+{ "l_orderkey": 1028, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24232.78, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic platelets. carefully f" }
+{ "l_orderkey": 1447, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5592.18, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-10", "l_receiptdate": "1992-11-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as! regular packages poach above the" }
 { "l_orderkey": 1475, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 30756.99, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1998-01-27", "l_receiptdate": "1998-01-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "quickly fluffy" }
+{ "l_orderkey": 1670, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38213.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely according to the sly" }
+{ "l_orderkey": 1762, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6524.21, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express packages wake slyly-- regul" }
+{ "l_orderkey": 2050, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10252.33, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ns. bold, final ideas cajole among the fi" }
+{ "l_orderkey": 2465, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7456.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s across the express deposits wak" }
+{ "l_orderkey": 2723, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-27", "l_commitdate": "1995-11-29", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al, special r" }
 { "l_orderkey": 2817, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4660.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-07", "l_commitdate": "1994-05-31", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "furiously unusual theodolites use furiou" }
 { "l_orderkey": 2885, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13980.45, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-31", "l_commitdate": "1992-11-24", "l_receiptdate": "1992-11-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "odolites. boldly pending packages han" }
-{ "l_orderkey": 3716, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. quickly sly ideas slee" }
-{ "l_orderkey": 4449, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. blithely final " }
-{ "l_orderkey": 5603, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 45669.47, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic, pending dependencies print" }
-{ "l_orderkey": 775, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 14912.48, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-23", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-06-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "un quickly slyly" }
-{ "l_orderkey": 2465, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7456.24, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-16", "l_commitdate": "1995-08-26", "l_receiptdate": "1995-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s across the express deposits wak" }
 { "l_orderkey": 3553, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 37281.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-14", "l_commitdate": "1994-06-26", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " slyly pending asymptotes against the furi" }
+{ "l_orderkey": 3716, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-02", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-12-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ts. quickly sly ideas slee" }
+{ "l_orderkey": 4038, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the furiously regu" }
+{ "l_orderkey": 4197, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26096.84, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular pin" }
+{ "l_orderkey": 4419, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. furious" }
+{ "l_orderkey": 4449, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-22", "l_commitdate": "1998-05-09", "l_receiptdate": "1998-04-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " packages. blithely final " }
+{ "l_orderkey": 4580, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-12-30", "l_receiptdate": "1994-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular, pending deposits. fina" }
 { "l_orderkey": 4770, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38213.23, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-04", "l_commitdate": "1995-08-08", "l_receiptdate": "1995-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ithely even packages sleep caref" }
+{ "l_orderkey": 4900, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18640.6, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-09-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yers. accounts affix somet" }
 { "l_orderkey": 4933, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 44737.44, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-10", "l_commitdate": "1995-10-03", "l_receiptdate": "1995-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ideas. sly" }
 { "l_orderkey": 5060, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26096.84, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-08-11", "l_receiptdate": "1992-10-09", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "c requests" }
 { "l_orderkey": 5127, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18640.6, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-11", "l_commitdate": "1997-02-26", "l_receiptdate": "1997-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "dolites about the final platelets w" }
-{ "l_orderkey": 1028, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24232.78, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-18", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ronic platelets. carefully f" }
-{ "l_orderkey": 1670, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38213.23, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-19", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely according to the sly" }
-{ "l_orderkey": 2050, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10252.33, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-27", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ns. bold, final ideas cajole among the fi" }
-{ "l_orderkey": 4197, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 26096.84, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-10-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "regular pin" }
-{ "l_orderkey": 4419, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 39145.26, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-18", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "sts. furious" }
-{ "l_orderkey": 4580, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-12-30", "l_receiptdate": "1994-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular, pending deposits. fina" }
 { "l_orderkey": 5249, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 12116.39, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-27", "l_commitdate": "1994-10-20", "l_receiptdate": "1994-10-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ites. finally exp" }
-{ "l_orderkey": 129, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-07", "l_commitdate": "1993-01-02", "l_receiptdate": "1992-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uests. foxes cajole slyly after the ca" }
-{ "l_orderkey": 640, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 41941.35, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-15", "l_commitdate": "1993-04-23", "l_receiptdate": "1993-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ong the qui" }
-{ "l_orderkey": 1447, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5592.18, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-24", "l_commitdate": "1992-12-10", "l_receiptdate": "1992-11-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "as! regular packages poach above the" }
-{ "l_orderkey": 1762, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6524.21, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-03", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-09-10", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "uickly express packages wake slyly-- regul" }
-{ "l_orderkey": 2723, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9320.3, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-27", "l_commitdate": "1995-11-29", "l_receiptdate": "1995-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "al, special r" }
-{ "l_orderkey": 4038, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22368.72, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-06", "l_commitdate": "1996-02-15", "l_receiptdate": "1996-04-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "the furiously regu" }
-{ "l_orderkey": 4900, "l_partkey": 32, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18640.6, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-09-23", "l_receiptdate": "1992-09-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "yers. accounts affix somet" }
-{ "l_orderkey": 1638, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "xcuses sleep furiou" }
-{ "l_orderkey": 5415, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14896.48, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-10-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans haggle furiously" }
+{ "l_orderkey": 5603, "l_partkey": 32, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 45669.47, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-07", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nic, pending dependencies print" }
 { "l_orderkey": 35, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26068.84, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ly alongside of " }
 { "l_orderkey": 354, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16758.54, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-31", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " about the carefully unusual " }
 { "l_orderkey": 484, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45620.47, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-06", "l_commitdate": "1997-02-28", "l_receiptdate": "1997-03-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ven accounts" }
 { "l_orderkey": 994, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-24", "l_commitdate": "1994-06-14", "l_receiptdate": "1994-06-26", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ainst the pending requests. packages sl" }
 { "l_orderkey": 1028, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 22.0, "l_extendedprice": 20482.66, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-24", "l_commitdate": "1994-02-27", "l_receiptdate": "1994-05-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " Tiresias alongside of the carefully spec" }
+{ "l_orderkey": 1185, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26068.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ke. slyly regular t" }
+{ "l_orderkey": 1476, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18620.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". bold deposits are carefully amo" }
+{ "l_orderkey": 1573, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-24", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully regular deposits. " }
+{ "l_orderkey": 1638, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-01", "l_receiptdate": "1997-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "xcuses sleep furiou" }
+{ "l_orderkey": 1797, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-07-11", "l_receiptdate": "1996-08-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " cajole carefully. unusual Tiresias e" }
 { "l_orderkey": 2403, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27930.9, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-08", "l_commitdate": "1998-06-17", "l_receiptdate": "1998-08-20", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ackages sleep furiously pendin" }
 { "l_orderkey": 2752, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38172.23, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-03-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "tructions hag" }
+{ "l_orderkey": 2753, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6517.21, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "xpress ideas detect b" }
+{ "l_orderkey": 3335, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-25", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r packages cajole ac" }
 { "l_orderkey": 3556, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 40034.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-18", "l_commitdate": "1992-11-09", "l_receiptdate": "1993-02-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "wake carefull" }
+{ "l_orderkey": 3781, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13965.45, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully blithe" }
 { "l_orderkey": 3808, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 41896.35, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-07-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " carefully special" }
 { "l_orderkey": 4197, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 48.0, "l_extendedprice": 44689.44, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-07", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final instructions. blithe, spe" }
+{ "l_orderkey": 4258, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42827.38, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously pend" }
 { "l_orderkey": 4355, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 46551.5, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1997-01-01", "l_receiptdate": "1996-12-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " regular accounts boost along the " }
+{ "l_orderkey": 4608, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 33517.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages wake quickly slyly iron" }
 { "l_orderkey": 4645, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 42.0, "l_extendedprice": 39103.26, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-31", "l_commitdate": "1994-10-22", "l_receiptdate": "1995-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "e slyly regular pinto beans. thin" }
+{ "l_orderkey": 4705, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13034.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ain carefully amon" }
 { "l_orderkey": 4864, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42827.38, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-24", "l_commitdate": "1993-01-02", "l_receiptdate": "1993-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "sts use carefully across the carefull" }
 { "l_orderkey": 4870, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4655.15, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-11", "l_commitdate": "1994-10-07", "l_receiptdate": "1994-10-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "s haggle furiously. slyly ironic dinos" }
+{ "l_orderkey": 5249, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-29", "l_receiptdate": "1994-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole furiousl" }
 { "l_orderkey": 5253, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32586.05, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-23", "l_commitdate": "1995-06-12", "l_receiptdate": "1995-08-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ven deposits. careful" }
 { "l_orderkey": 5285, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34448.11, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-26", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-03-27", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uffily regu" }
-{ "l_orderkey": 1476, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18620.6, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-11", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-08-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". bold deposits are carefully amo" }
-{ "l_orderkey": 1797, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-06", "l_commitdate": "1996-07-11", "l_receiptdate": "1996-08-29", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " cajole carefully. unusual Tiresias e" }
-{ "l_orderkey": 3335, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-05", "l_commitdate": "1995-12-25", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r packages cajole ac" }
 { "l_orderkey": 5413, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 29792.96, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-23", "l_commitdate": "1997-12-09", "l_receiptdate": "1997-11-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he quickly ironic ideas. slyly ironic ide" }
+{ "l_orderkey": 5415, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14896.48, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-29", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-10-10", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "pinto beans haggle furiously" }
 { "l_orderkey": 5575, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21413.69, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-10-09", "l_receiptdate": "1995-11-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "enticingly final requests. ironically" }
-{ "l_orderkey": 1185, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26068.84, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-24", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-10-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ke. slyly regular t" }
-{ "l_orderkey": 1573, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15827.51, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-24", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully regular deposits. " }
-{ "l_orderkey": 2753, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6517.21, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-11", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-03-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "xpress ideas detect b" }
-{ "l_orderkey": 3781, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13965.45, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-23", "l_commitdate": "1996-08-08", "l_receiptdate": "1996-09-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " carefully blithe" }
-{ "l_orderkey": 4258, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42827.38, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-01-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " furiously pend" }
-{ "l_orderkey": 4608, "l_partkey": 31, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 33517.08, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-04", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-10-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ages wake quickly slyly iron" }
-{ "l_orderkey": 4705, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 13034.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-07-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ain carefully amon" }
-{ "l_orderkey": 5249, "l_partkey": 31, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40965.32, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-11-29", "l_receiptdate": "1994-12-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ole furiousl" }
+{ "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1860.06, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
+{ "l_orderkey": 103, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29760.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-30", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-08-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages doze. special, regular deposit" }
+{ "l_orderkey": 646, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22320.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-20", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "regular accounts haggle dog" }
+{ "l_orderkey": 1121, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 43711.41, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly idle, i" }
 { "l_orderkey": 1282, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9300.3, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-10", "l_commitdate": "1992-04-16", "l_receiptdate": "1992-05-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "r theodolite" }
+{ "l_orderkey": 1702, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35341.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "as believe blithely. bo" }
+{ "l_orderkey": 1862, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38131.23, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully along" }
 { "l_orderkey": 1925, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-04-06", "l_receiptdate": "1992-06-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "instructions sleep. pinto bea" }
+{ "l_orderkey": 2208, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 39991.29, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "es. accounts cajole. fi" }
 { "l_orderkey": 2339, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 26040.84, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-25", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-01-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e bold, even packag" }
 { "l_orderkey": 2533, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-23", "l_commitdate": "1997-05-10", "l_receiptdate": "1997-06-18", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ackages. blith" }
-{ "l_orderkey": 5986, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 930.03, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-21", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fix quickly quickly final deposits. fluffil" }
-{ "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1860.06, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
-{ "l_orderkey": 1702, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 35341.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-01", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "as believe blithely. bo" }
-{ "l_orderkey": 2208, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 39991.29, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-10", "l_commitdate": "1995-06-02", "l_receiptdate": "1995-06-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "es. accounts cajole. fi" }
 { "l_orderkey": 2662, "l_partkey": 30, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31621.02, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-10-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ding theodolites use carefully. p" }
 { "l_orderkey": 3200, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10230.33, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-18", "l_commitdate": "1996-03-21", "l_receiptdate": "1996-04-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "osits sleep fur" }
+{ "l_orderkey": 3655, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32551.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-11-16", "l_receiptdate": "1993-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "blithely even accounts! furiously regular" }
+{ "l_orderkey": 3714, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 40921.32, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-18", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. quickly ironic dugouts sublat" }
 { "l_orderkey": 3781, "l_partkey": 30, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-23", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". theodolite" }
 { "l_orderkey": 3970, "l_partkey": 30, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21390.69, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " above the final braids. regular" }
 { "l_orderkey": 3973, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19530.63, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-18", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "equests. furiously" }
 { "l_orderkey": 5025, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10230.33, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-21", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-03-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the carefully final esc" }
 { "l_orderkey": 5377, "l_partkey": 30, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15810.51, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-05", "l_commitdate": "1997-05-25", "l_receiptdate": "1997-07-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "dencies. carefully regular re" }
-{ "l_orderkey": 103, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29760.96, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-30", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-08-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "kages doze. special, regular deposit" }
-{ "l_orderkey": 1862, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 38131.23, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-05", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-07-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully along" }
 { "l_orderkey": 5891, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9300.3, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-02-07", "l_receiptdate": "1993-04-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nding requests. b" }
-{ "l_orderkey": 646, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22320.72, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-20", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "regular accounts haggle dog" }
-{ "l_orderkey": 1121, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 43711.41, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-27", "l_commitdate": "1997-03-28", "l_receiptdate": "1997-05-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ly idle, i" }
-{ "l_orderkey": 3655, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32551.05, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1992-11-16", "l_receiptdate": "1993-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "blithely even accounts! furiously regular" }
-{ "l_orderkey": 3714, "l_partkey": 30, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 40921.32, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-18", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-07-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. quickly ironic dugouts sublat" }
+{ "l_orderkey": 5986, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 930.03, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-06-21", "l_receiptdate": "1992-05-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "fix quickly quickly final deposits. fluffil" }
 { "l_orderkey": 103, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-11", "l_commitdate": "1996-09-18", "l_receiptdate": "1996-09-26", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ironic accou" }
-{ "l_orderkey": 868, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12077.26, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-25", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "gged instructi" }
-{ "l_orderkey": 1059, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6503.14, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furiously silent excuses are e" }
-{ "l_orderkey": 3298, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23225.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-31", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly express f" }
-{ "l_orderkey": 3591, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19509.42, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "structions against " }
-{ "l_orderkey": 3778, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-22", "l_commitdate": "1993-08-18", "l_receiptdate": "1993-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tes affix carefully above the " }
-{ "l_orderkey": 4161, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-11-17", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "he stealthily ironic foxes. ideas haggl" }
-{ "l_orderkey": 4647, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26012.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " are above the fluffily fin" }
-{ "l_orderkey": 4738, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld, even packages. furio" }
+{ "l_orderkey": 132, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-08-27", "l_receiptdate": "1993-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "refully blithely bold acco" }
+{ "l_orderkey": 358, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16722.36, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olphins haggle ironic accounts. f" }
 { "l_orderkey": 486, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2787.06, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-07", "l_commitdate": "1996-04-20", "l_receiptdate": "1996-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ccounts ha" }
-{ "l_orderkey": 1575, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39018.84, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly pending pinto beans." }
-{ "l_orderkey": 2658, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts cajole. pending packages affix" }
-{ "l_orderkey": 3205, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lar accoun" }
-{ "l_orderkey": 3489, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-02", "l_commitdate": "1993-10-09", "l_receiptdate": "1993-08-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "xcuses? quickly stealthy dependenci" }
-{ "l_orderkey": 3558, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35302.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully permanently iron" }
-{ "l_orderkey": 4070, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nticing ideas. boldly" }
-{ "l_orderkey": 4263, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5574.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g the final, regular instructions: " }
-{ "l_orderkey": 4295, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45521.98, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "refully silent requests. f" }
 { "l_orderkey": 773, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 26012.56, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1993-11-05", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he furiously slow deposits." }
 { "l_orderkey": 838, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 25083.54, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-15", "l_commitdate": "1998-04-03", "l_receiptdate": "1998-02-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " pending pinto beans haggle about t" }
+{ "l_orderkey": 868, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 12077.26, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-25", "l_commitdate": "1992-08-26", "l_receiptdate": "1992-08-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "gged instructi" }
 { "l_orderkey": 1028, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 25083.54, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-03", "l_commitdate": "1994-02-07", "l_receiptdate": "1994-04-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ial accounts nag. slyly" }
+{ "l_orderkey": 1059, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6503.14, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-30", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "the furiously silent excuses are e" }
+{ "l_orderkey": 1575, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 39018.84, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-21", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-10-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ly pending pinto beans." }
+{ "l_orderkey": 1956, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10219.22, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-10-29", "l_receiptdate": "1993-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " the braids slee" }
 { "l_orderkey": 2147, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46451.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-11-30", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "al accounts. even, even foxes wake" }
+{ "l_orderkey": 2630, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole. e" }
+{ "l_orderkey": 2658, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-12", "l_commitdate": "1995-11-18", "l_receiptdate": "1995-11-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts cajole. pending packages affix" }
+{ "l_orderkey": 2784, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2787.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "n packages. foxes haggle quickly sile" }
 { "l_orderkey": 2791, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24154.52, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-06", "l_commitdate": "1994-12-07", "l_receiptdate": "1995-02-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uriously special instructio" }
+{ "l_orderkey": 2978, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 30657.66, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. blithely unusual pack" }
+{ "l_orderkey": 3073, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13006.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ilently quiet epitaphs." }
+{ "l_orderkey": 3205, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-07-10", "l_receiptdate": "1992-06-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lar accoun" }
+{ "l_orderkey": 3298, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23225.5, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-31", "l_receiptdate": "1996-07-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ly express f" }
+{ "l_orderkey": 3489, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-02", "l_commitdate": "1993-10-09", "l_receiptdate": "1993-08-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "xcuses? quickly stealthy dependenci" }
+{ "l_orderkey": 3558, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35302.76, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-29", "l_commitdate": "1996-05-02", "l_receiptdate": "1996-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully permanently iron" }
+{ "l_orderkey": 3591, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19509.42, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-25", "l_commitdate": "1994-02-02", "l_receiptdate": "1994-03-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "structions against " }
+{ "l_orderkey": 3778, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29728.64, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-22", "l_commitdate": "1993-08-18", "l_receiptdate": "1993-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tes affix carefully above the " }
+{ "l_orderkey": 4070, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-22", "l_commitdate": "1995-07-14", "l_receiptdate": "1995-07-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "nticing ideas. boldly" }
+{ "l_orderkey": 4161, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-11-17", "l_receiptdate": "1993-11-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "he stealthily ironic foxes. ideas haggl" }
+{ "l_orderkey": 4263, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 6.0, "l_extendedprice": 5574.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-01", "l_commitdate": "1998-06-02", "l_receiptdate": "1998-05-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g the final, regular instructions: " }
+{ "l_orderkey": 4295, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45521.98, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-25", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "refully silent requests. f" }
+{ "l_orderkey": 4647, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 28.0, "l_extendedprice": 26012.56, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-25", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " are above the fluffily fin" }
+{ "l_orderkey": 4738, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 20438.44, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-25", "l_commitdate": "1992-05-19", "l_receiptdate": "1992-06-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld, even packages. furio" }
 { "l_orderkey": 4806, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7432.16, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-08", "l_commitdate": "1993-07-16", "l_receiptdate": "1993-05-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "requests boost blithely. qui" }
 { "l_orderkey": 4997, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1858.04, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-09", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "counts. slyl" }
 { "l_orderkey": 5254, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-16", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-09-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "lyly regular accounts. furiously pendin" }
 { "l_orderkey": 5410, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 37160.8, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-17", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-11-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "iously special accounts are along th" }
 { "l_orderkey": 5569, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23225.5, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-07-18", "l_receiptdate": "1993-07-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " deposits cajole above" }
-{ "l_orderkey": 132, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21367.46, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-16", "l_commitdate": "1993-08-27", "l_receiptdate": "1993-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "refully blithely bold acco" }
-{ "l_orderkey": 358, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16722.36, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-07", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "olphins haggle ironic accounts. f" }
-{ "l_orderkey": 1956, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10219.22, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-19", "l_commitdate": "1992-10-29", "l_receiptdate": "1993-01-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " the braids slee" }
-{ "l_orderkey": 2630, "l_partkey": 29, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42734.92, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole. e" }
-{ "l_orderkey": 2784, "l_partkey": 29, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2787.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-19", "l_commitdate": "1998-04-05", "l_receiptdate": "1998-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "n packages. foxes haggle quickly sile" }
-{ "l_orderkey": 2978, "l_partkey": 29, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 33.0, "l_extendedprice": 30657.66, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-06", "l_commitdate": "1995-07-23", "l_receiptdate": "1995-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. blithely unusual pack" }
-{ "l_orderkey": 3073, "l_partkey": 29, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 13006.28, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-24", "l_commitdate": "1994-04-01", "l_receiptdate": "1994-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ilently quiet epitaphs." }
-{ "l_orderkey": 485, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37120.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al escapades" }
-{ "l_orderkey": 935, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21344.46, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular accounts about" }
-{ "l_orderkey": 2849, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-08", "l_receiptdate": "1996-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. carefully silent" }
-{ "l_orderkey": 4034, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42688.92, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-22", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests. furiously unusual instructions wake" }
-{ "l_orderkey": 4039, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39904.86, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-01-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans believe bene" }
-{ "l_orderkey": 4704, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6496.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ers wake car" }
-{ "l_orderkey": 5699, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 19488.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly final pla" }
 { "l_orderkey": 455, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40832.88, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-02-22", "l_receiptdate": "1997-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " accounts sleep slyly ironic asymptote" }
-{ "l_orderkey": 2240, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34336.74, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-16", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly after the packages? blithely si" }
-{ "l_orderkey": 3520, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "deas should solve blithely among the ironi" }
-{ "l_orderkey": 3555, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 30624.66, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily regular a" }
-{ "l_orderkey": 3746, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10208.22, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic theodolites are among th" }
-{ "l_orderkey": 5921, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24128.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hy dependenc" }
+{ "l_orderkey": 485, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 37120.8, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-29", "l_commitdate": "1997-05-08", "l_receiptdate": "1997-04-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al escapades" }
 { "l_orderkey": 645, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 8352.18, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-25", "l_commitdate": "1995-01-04", "l_receiptdate": "1995-01-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "special deposits. regular, final th" }
+{ "l_orderkey": 935, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21344.46, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-11-22", "l_receiptdate": "1997-11-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ular accounts about" }
+{ "l_orderkey": 1445, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15776.34, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ges. furiously regular pint" }
 { "l_orderkey": 1506, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34336.74, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-04", "l_commitdate": "1992-12-01", "l_receiptdate": "1992-11-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "carefully bold dolphins. accounts su" }
 { "l_orderkey": 1509, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12992.28, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-09-25", "l_receiptdate": "1993-10-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nal realms" }
 { "l_orderkey": 1985, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30624.66, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-11-01", "l_receiptdate": "1994-12-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "s are express packages. pendin" }
-{ "l_orderkey": 3495, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18560.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "posits are carefully; forges cajole qui" }
-{ "l_orderkey": 4161, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43616.94, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-10-29", "l_receiptdate": "1994-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r requests about the final, even foxes hag" }
-{ "l_orderkey": 1445, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 17.0, "l_extendedprice": 15776.34, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-02", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ges. furiously regular pint" }
+{ "l_orderkey": 2240, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34336.74, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-16", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " quickly after the packages? blithely si" }
+{ "l_orderkey": 2849, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-24", "l_commitdate": "1996-07-08", "l_receiptdate": "1996-09-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly. carefully silent" }
 { "l_orderkey": 2978, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6496.14, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-18", "l_commitdate": "1995-07-03", "l_receiptdate": "1995-07-23", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". final ideas are blithe" }
+{ "l_orderkey": 3495, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18560.4, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-05-18", "l_receiptdate": "1996-05-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "posits are carefully; forges cajole qui" }
+{ "l_orderkey": 3520, "l_partkey": 28, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27840.6, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-11", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "deas should solve blithely among the ironi" }
+{ "l_orderkey": 3555, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 30624.66, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-20", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily regular a" }
+{ "l_orderkey": 3746, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10208.22, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-02", "l_commitdate": "1994-11-19", "l_receiptdate": "1994-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " ironic theodolites are among th" }
+{ "l_orderkey": 4034, "l_partkey": 28, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42688.92, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-22", "l_commitdate": "1994-01-09", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uests. furiously unusual instructions wake" }
+{ "l_orderkey": 4039, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39904.86, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-02", "l_commitdate": "1997-12-22", "l_receiptdate": "1998-01-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "beans believe bene" }
+{ "l_orderkey": 4161, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43616.94, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-20", "l_commitdate": "1993-10-29", "l_receiptdate": "1994-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "r requests about the final, even foxes hag" }
+{ "l_orderkey": 4704, "l_partkey": 28, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 7.0, "l_extendedprice": 6496.14, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ers wake car" }
+{ "l_orderkey": 5699, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 21.0, "l_extendedprice": 19488.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-09-30", "l_receiptdate": "1992-10-19", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lyly final pla" }
+{ "l_orderkey": 5921, "l_partkey": 28, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 24128.52, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-05-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "hy dependenc" }
 { "l_orderkey": 448, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32445.7, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-10-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ses nag quickly quickly ir" }
-{ "l_orderkey": 2084, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25956.56, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cajole quickly carefu" }
-{ "l_orderkey": 3783, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-02-17", "l_receiptdate": "1993-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing to the ideas. regular accounts de" }
-{ "l_orderkey": 1411, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-03-02", "l_receiptdate": "1995-03-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d excuses. furiously final pear" }
-{ "l_orderkey": 1861, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "arefully unusual" }
-{ "l_orderkey": 2534, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45423.98, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-20", "l_receiptdate": "1996-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sometimes regular requests. blithely unus" }
-{ "l_orderkey": 3782, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26883.58, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly unusual pinto beans. carefully fina" }
-{ "l_orderkey": 4294, "l_partkey": 27, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14832.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lithely pint" }
-{ "l_orderkey": 2405, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27810.6, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y final deposits are slyly caref" }
-{ "l_orderkey": 4096, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-09-03", "l_receiptdate": "1992-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y final, even platelets. boldly" }
-{ "l_orderkey": 4129, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36153.78, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-10-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y regular foxes. slyly ironic deposits " }
-{ "l_orderkey": 4544, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7416.16, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "olites. fi" }
 { "l_orderkey": 1124, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 43.0, "l_extendedprice": 39861.86, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-10-28", "l_receiptdate": "1998-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "across the " }
+{ "l_orderkey": 1411, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-27", "l_commitdate": "1995-03-02", "l_receiptdate": "1995-03-24", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "d excuses. furiously final pear" }
 { "l_orderkey": 1728, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31518.68, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-20", "l_receiptdate": "1996-09-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "special req" }
 { "l_orderkey": 1799, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 38934.84, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-05", "l_commitdate": "1994-04-28", "l_receiptdate": "1994-04-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "es pending " }
 { "l_orderkey": 1826, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3708.08, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-05", "l_commitdate": "1992-06-12", "l_receiptdate": "1992-08-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "alongside of the quickly unusual re" }
+{ "l_orderkey": 1861, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-29", "l_commitdate": "1994-03-07", "l_receiptdate": "1994-02-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "arefully unusual" }
+{ "l_orderkey": 2084, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25956.56, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-04", "l_commitdate": "1993-05-14", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "cajole quickly carefu" }
 { "l_orderkey": 2342, "l_partkey": 27, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20394.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-10", "l_commitdate": "1996-08-02", "l_receiptdate": "1996-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s. ironic " }
+{ "l_orderkey": 2405, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27810.6, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-03-10", "l_receiptdate": "1997-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y final deposits are slyly caref" }
+{ "l_orderkey": 2534, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45423.98, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-01", "l_commitdate": "1996-08-20", "l_receiptdate": "1996-09-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sometimes regular requests. blithely unus" }
+{ "l_orderkey": 3782, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26883.58, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-10-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "quickly unusual pinto beans. carefully fina" }
+{ "l_orderkey": 3783, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34299.74, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-02-17", "l_receiptdate": "1993-12-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing to the ideas. regular accounts de" }
+{ "l_orderkey": 4096, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28737.62, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-14", "l_commitdate": "1992-09-03", "l_receiptdate": "1992-07-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "y final, even platelets. boldly" }
+{ "l_orderkey": 4129, "l_partkey": 27, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 36153.78, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-21", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-10-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y regular foxes. slyly ironic deposits " }
+{ "l_orderkey": 4294, "l_partkey": 27, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14832.32, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-09-24", "l_receiptdate": "1992-09-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lithely pint" }
+{ "l_orderkey": 4544, "l_partkey": 27, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 8.0, "l_extendedprice": 7416.16, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-13", "l_commitdate": "1997-10-06", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "olites. fi" }
 { "l_orderkey": 4640, "l_partkey": 27, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16686.36, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-06", "l_receiptdate": "1996-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "boost furiously accord" }
 { "l_orderkey": 453, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 29632.64, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-15", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "s. fluffily bold packages cajole. unu" }
-{ "l_orderkey": 2567, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36114.78, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-10", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns. furiously final dependencies cajo" }
-{ "l_orderkey": 3527, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 30558.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly alongside of " }
-{ "l_orderkey": 4800, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-23", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nal accounts are blithely deposits. bol" }
-{ "l_orderkey": 801, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10186.22, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y special pinto beans cajole " }
-{ "l_orderkey": 1089, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-24", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "g dolphins. deposits integrate. s" }
-{ "l_orderkey": 2758, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ake furious" }
-{ "l_orderkey": 2884, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "pending accounts about " }
-{ "l_orderkey": 3968, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 41670.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-18", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully slyly fi" }
-{ "l_orderkey": 4007, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ter the accounts. expr" }
-{ "l_orderkey": 4646, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20372.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cies are blithely after the slyly reg" }
 { "l_orderkey": 577, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23150.5, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-09", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-05-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ve slyly of the frets. careful" }
+{ "l_orderkey": 801, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10186.22, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-09", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-05-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y special pinto beans cajole " }
 { "l_orderkey": 961, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35188.76, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-21", "l_commitdate": "1995-07-19", "l_receiptdate": "1995-08-27", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "he blithely special requests. furiousl" }
-{ "l_orderkey": 1442, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "c deposits haggle after the even" }
-{ "l_orderkey": 4801, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31484.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-02-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final requests " }
-{ "l_orderkey": 5027, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34262.74, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-10-30", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ost slyly fluffily" }
-{ "l_orderkey": 5123, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12038.26, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "regular pearls" }
 { "l_orderkey": 1028, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 36114.78, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-27", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-03-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodoli" }
+{ "l_orderkey": 1089, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-24", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-07-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "g dolphins. deposits integrate. s" }
+{ "l_orderkey": 1442, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-11-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "c deposits haggle after the even" }
 { "l_orderkey": 1541, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-08-07", "l_receiptdate": "1995-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y pending packages. blithely fi" }
 { "l_orderkey": 1762, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13890.3, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-18", "l_commitdate": "1994-10-29", "l_receiptdate": "1995-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "old packages thrash. care" }
 { "l_orderkey": 2146, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28706.62, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-04", "l_commitdate": "1992-10-24", "l_receiptdate": "1993-01-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "lly even deposit" }
+{ "l_orderkey": 2567, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 36114.78, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-10", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-05-21", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns. furiously final dependencies cajo" }
+{ "l_orderkey": 2758, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-09", "l_commitdate": "1998-09-15", "l_receiptdate": "1998-10-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ake furious" }
+{ "l_orderkey": 2884, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7408.16, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1997-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "pending accounts about " }
+{ "l_orderkey": 3527, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 30558.66, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-25", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-10-12", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "kly alongside of " }
+{ "l_orderkey": 3968, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 41670.9, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-18", "l_commitdate": "1997-04-24", "l_receiptdate": "1997-06-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ully slyly fi" }
+{ "l_orderkey": 4007, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21298.46, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-08", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-10-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ter the accounts. expr" }
 { "l_orderkey": 4131, "l_partkey": 26, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23150.5, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-24", "l_commitdate": "1998-03-01", "l_receiptdate": "1998-02-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uickly exp" }
+{ "l_orderkey": 4646, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20372.44, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-14", "l_commitdate": "1996-08-06", "l_receiptdate": "1996-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "cies are blithely after the slyly reg" }
+{ "l_orderkey": 4800, "l_partkey": 26, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 926.02, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-23", "l_commitdate": "1992-03-16", "l_receiptdate": "1992-03-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "nal accounts are blithely deposits. bol" }
+{ "l_orderkey": 4801, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31484.68, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-02-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "y final requests " }
+{ "l_orderkey": 5027, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34262.74, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-05", "l_commitdate": "1997-10-30", "l_receiptdate": "1997-10-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ost slyly fluffily" }
+{ "l_orderkey": 5123, "l_partkey": 26, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 12038.26, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-17", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "regular pearls" }
 { "l_orderkey": 5572, "l_partkey": 26, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 24.0, "l_extendedprice": 22224.48, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-26", "l_commitdate": "1994-09-04", "l_receiptdate": "1994-10-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " beans. foxes sleep fluffily across th" }
 { "l_orderkey": 1, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24.0, "l_extendedprice": 22200.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending foxes. slyly re" }
 { "l_orderkey": 516, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10175.22, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ongside of the blithely final reque" }
 { "l_orderkey": 768, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34225.74, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-09-23", "l_receiptdate": "1996-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ending requests across the quickly" }
+{ "l_orderkey": 868, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 24975.54, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "oss the fluffily unusual pinto " }
 { "l_orderkey": 1441, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 15.0, "l_extendedprice": 13875.3, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-21", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-06-04", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " dependencies-- cour" }
-{ "l_orderkey": 2688, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2775.06, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-04", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffily " }
-{ "l_orderkey": 2980, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45325.98, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-10-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hy packages sleep quic" }
-{ "l_orderkey": 3523, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13875.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se slyly pending, sp" }
-{ "l_orderkey": 5505, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y alongside of the special requests." }
-{ "l_orderkey": 5762, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25900.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-22", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic foxes among the blithely qui" }
+{ "l_orderkey": 1508, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18500.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nic platelets. carefully final fra" }
 { "l_orderkey": 1540, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5550.12, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-28", "l_commitdate": "1992-09-17", "l_receiptdate": "1992-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ing to the slyly express asymptote" }
+{ "l_orderkey": 1763, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14800.32, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ously pending asymptotes a" }
+{ "l_orderkey": 1767, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the bravely ironic requests i" }
+{ "l_orderkey": 1793, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38850.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-11-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uctions sleep carefully special, fl" }
+{ "l_orderkey": 1830, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8325.18, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-03-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st furiously among " }
+{ "l_orderkey": 2051, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ounts sleep fluffily even requ" }
 { "l_orderkey": 2146, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 12950.28, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-16", "l_commitdate": "1992-10-16", "l_receiptdate": "1992-09-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ecial, express a" }
 { "l_orderkey": 2146, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 36075.78, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-11-06", "l_receiptdate": "1993-01-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "uickly regular excuses detect. regular c" }
 { "l_orderkey": 2561, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1997-12-28", "l_receiptdate": "1998-01-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "bold packages wake slyly. slyly" }
+{ "l_orderkey": 2688, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2775.06, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-02-04", "l_commitdate": "1992-03-18", "l_receiptdate": "1992-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e fluffily " }
+{ "l_orderkey": 2980, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45325.98, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-10-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hy packages sleep quic" }
 { "l_orderkey": 3105, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 44400.96, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-28", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-03-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ending platelets wake carefully ironic inst" }
+{ "l_orderkey": 3265, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7400.16, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely ironic requests sleep slyly-- i" }
+{ "l_orderkey": 3523, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13875.3, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-26", "l_commitdate": "1998-05-22", "l_receiptdate": "1998-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "se slyly pending, sp" }
 { "l_orderkey": 3585, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 12025.26, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-15", "l_commitdate": "1995-01-22", "l_receiptdate": "1995-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ccording to the foxes. slyly iro" }
 { "l_orderkey": 4709, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 23125.5, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "deposits grow. fluffily unusual accounts " }
-{ "l_orderkey": 868, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 24975.54, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-01", "l_commitdate": "1992-08-25", "l_receiptdate": "1992-08-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "oss the fluffily unusual pinto " }
-{ "l_orderkey": 1508, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18500.4, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-11", "l_receiptdate": "1998-05-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "nic platelets. carefully final fra" }
-{ "l_orderkey": 1767, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-22", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "to the bravely ironic requests i" }
-{ "l_orderkey": 1793, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38850.84, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-13", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-11-06", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uctions sleep carefully special, fl" }
-{ "l_orderkey": 3265, "l_partkey": 25, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7400.16, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-01", "l_commitdate": "1992-09-12", "l_receiptdate": "1992-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "thely ironic requests sleep slyly-- i" }
 { "l_orderkey": 5060, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 24975.54, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-09-05", "l_receiptdate": "1992-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. ironic " }
 { "l_orderkey": 5185, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29600.64, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-09-30", "l_receiptdate": "1997-08-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ackages. slyly even requests" }
-{ "l_orderkey": 1763, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14800.32, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-12-04", "l_receiptdate": "1996-12-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ously pending asymptotes a" }
-{ "l_orderkey": 1830, "l_partkey": 25, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8325.18, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-05-24", "l_receiptdate": "1995-03-14", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "st furiously among " }
-{ "l_orderkey": 2051, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-06-16", "l_receiptdate": "1996-04-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ounts sleep fluffily even requ" }
+{ "l_orderkey": 5505, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 39775.86, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-30", "l_commitdate": "1997-11-28", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y alongside of the special requests." }
+{ "l_orderkey": 5762, "l_partkey": 25, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25900.56, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-22", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ic foxes among the blithely qui" }
 { "l_orderkey": 259, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 38808.84, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-20", "l_commitdate": "1993-11-18", "l_receiptdate": "1993-11-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the slyly ironic pinto beans. fi" }
-{ "l_orderkey": 551, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7392.16, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly slyly pending platel" }
-{ "l_orderkey": 1733, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "slyly express deposits sleep abo" }
-{ "l_orderkey": 2595, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17556.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-23", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ns are neve" }
-{ "l_orderkey": 3623, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-01-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress ideas are furio" }
-{ "l_orderkey": 3845, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely bold ideas use. ex" }
 { "l_orderkey": 263, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20328.44, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-24", "l_commitdate": "1994-06-20", "l_receiptdate": "1994-09-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "efully express fo" }
 { "l_orderkey": 517, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21252.46, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-19", "l_commitdate": "1997-05-07", "l_receiptdate": "1997-05-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " kindle. furiously bold requests mus" }
+{ "l_orderkey": 549, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35112.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-23", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits. carefully regular depos" }
+{ "l_orderkey": 551, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7392.16, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-29", "l_commitdate": "1995-07-18", "l_receiptdate": "1995-08-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake quickly slyly pending platel" }
 { "l_orderkey": 677, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42504.92, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-02", "l_commitdate": "1994-02-12", "l_receiptdate": "1993-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ng theodolites. furiously unusual theodo" }
 { "l_orderkey": 995, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16632.36, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-03", "l_commitdate": "1995-07-29", "l_receiptdate": "1995-07-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " even accounts unwind c" }
 { "l_orderkey": 1572, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 37884.82, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-16", "l_commitdate": "1996-04-09", "l_receiptdate": "1996-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". pinto beans alongside" }
+{ "l_orderkey": 1733, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-07-25", "l_receiptdate": "1996-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "slyly express deposits sleep abo" }
+{ "l_orderkey": 1761, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11088.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " sleep furiously. deposits are acco" }
+{ "l_orderkey": 1861, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21252.46, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "in packages sleep silent dolphins; sly" }
 { "l_orderkey": 2083, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 34188.74, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-09-30", "l_receiptdate": "1993-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ng the special foxes wake packages. f" }
 { "l_orderkey": 2369, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27720.6, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-23", "l_commitdate": "1997-02-12", "l_receiptdate": "1997-05-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pecial deposits sleep. blithely unusual w" }
+{ "l_orderkey": 2496, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27720.6, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake. ironic foxes cajole quickly. fu" }
+{ "l_orderkey": 2595, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17556.38, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-23", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-01-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ns are neve" }
 { "l_orderkey": 2752, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36960.8, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-02-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "into beans are after the sly" }
+{ "l_orderkey": 2755, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10164.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular excuses sleep carefully." }
+{ "l_orderkey": 3008, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36960.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "esias. theodolites detect blithely " }
+{ "l_orderkey": 3010, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25872.56, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-03-28", "l_receiptdate": "1996-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ake carefully carefully even request" }
 { "l_orderkey": 3079, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 1848.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-05", "l_commitdate": "1997-11-17", "l_receiptdate": "1998-01-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ly busy requests believ" }
 { "l_orderkey": 3584, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5544.12, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-28", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "deposits across the" }
-{ "l_orderkey": 4192, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29568.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-23", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-07-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ounts are fluffily slyly bold req" }
-{ "l_orderkey": 4260, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-06", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "al, pending accounts must" }
-{ "l_orderkey": 549, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 35112.76, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-23", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-08-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "eposits. carefully regular depos" }
-{ "l_orderkey": 1761, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 11088.24, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-16", "l_commitdate": "1994-03-08", "l_receiptdate": "1994-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " sleep furiously. deposits are acco" }
-{ "l_orderkey": 2755, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10164.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-07", "l_receiptdate": "1992-04-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "egular excuses sleep carefully." }
+{ "l_orderkey": 3623, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-19", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-01-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress ideas are furio" }
+{ "l_orderkey": 3845, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 16.0, "l_extendedprice": 14784.32, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-08", "l_commitdate": "1992-06-08", "l_receiptdate": "1992-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ely bold ideas use. ex" }
 { "l_orderkey": 4006, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13860.3, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-04-02", "l_receiptdate": "1995-02-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n deposits cajole slyl" }
 { "l_orderkey": 4133, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32340.7, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-09-15", "l_receiptdate": "1992-12-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "g above the quickly bold packages. ev" }
 { "l_orderkey": 4166, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 26.0, "l_extendedprice": 24024.52, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-17", "l_commitdate": "1993-05-09", "l_receiptdate": "1993-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lar dependencies. s" }
+{ "l_orderkey": 4192, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29568.64, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-23", "l_commitdate": "1998-06-25", "l_receiptdate": "1998-07-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ounts are fluffily slyly bold req" }
 { "l_orderkey": 4224, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3696.08, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-09-05", "l_receiptdate": "1997-09-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " even dinos. carefull" }
-{ "l_orderkey": 5510, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26796.58, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely fluffily ironic req" }
-{ "l_orderkey": 1861, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 21252.46, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-09", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-04-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "in packages sleep silent dolphins; sly" }
-{ "l_orderkey": 2496, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27720.6, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-27", "l_commitdate": "1994-03-11", "l_receiptdate": "1994-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ake. ironic foxes cajole quickly. fu" }
-{ "l_orderkey": 3008, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36960.8, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-18", "l_commitdate": "1996-01-06", "l_receiptdate": "1996-01-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "esias. theodolites detect blithely " }
-{ "l_orderkey": 3010, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25872.56, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-05", "l_commitdate": "1996-03-28", "l_receiptdate": "1996-04-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ake carefully carefully even request" }
+{ "l_orderkey": 4260, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19404.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-06", "l_commitdate": "1992-06-18", "l_receiptdate": "1992-08-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "al, pending accounts must" }
 { "l_orderkey": 4261, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25872.56, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-08", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-10-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "packages. fluffily i" }
 { "l_orderkey": 4481, "l_partkey": 24, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 46201.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-08-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ar packages. regula" }
 { "l_orderkey": 5061, "l_partkey": 24, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 24024.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-07", "l_commitdate": "1993-09-13", "l_receiptdate": "1993-11-13", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " cajole slyly. carefully spe" }
 { "l_orderkey": 5126, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30492.66, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-04", "l_commitdate": "1992-12-23", "l_receiptdate": "1993-02-14", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ipliers promise furiously whithout the " }
+{ "l_orderkey": 5510, "l_partkey": 24, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26796.58, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-03-28", "l_receiptdate": "1993-03-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely fluffily ironic req" }
 { "l_orderkey": 5572, "l_partkey": 24, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 34.0, "l_extendedprice": 31416.68, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-22", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-11-08", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "asymptotes integrate. s" }
 { "l_orderkey": 37, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36920.8, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-21", "l_commitdate": "1992-08-01", "l_receiptdate": "1992-08-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "luffily regular requests. slyly final acco" }
 { "l_orderkey": 512, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 34151.74, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-20", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-07-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "nic depths cajole? blithely b" }
+{ "l_orderkey": 708, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6461.14, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-16", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lly express ac" }
 { "l_orderkey": 1025, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 23075.5, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-06-21", "l_receiptdate": "1995-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "xpress foxes. furiousl" }
-{ "l_orderkey": 2758, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15691.34, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-25", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts! qui" }
-{ "l_orderkey": 4609, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42458.92, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "r foxes. fluffily ironic ideas ha" }
 { "l_orderkey": 1282, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12922.28, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-29", "l_commitdate": "1992-04-05", "l_receiptdate": "1992-07-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ecial deposit" }
-{ "l_orderkey": 1856, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 33228.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly even foxes kindle blithely even realm" }
-{ "l_orderkey": 2209, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36920.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ully special sheaves serve" }
-{ "l_orderkey": 2566, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16614.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1992-12-24", "l_receiptdate": "1992-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " braids according t" }
-{ "l_orderkey": 5987, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 923.02, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "refully final excuses haggle furiously ag" }
+{ "l_orderkey": 1382, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ter the carefully final excuses. blit" }
 { "l_orderkey": 1412, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1846.04, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-01", "l_commitdate": "1993-05-03", "l_receiptdate": "1993-04-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s among the requests are a" }
 { "l_orderkey": 1767, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 46151.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-05-29", "l_commitdate": "1995-04-14", "l_receiptdate": "1995-06-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "y unusual foxe" }
+{ "l_orderkey": 1856, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 36.0, "l_extendedprice": 33228.72, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-19", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-06-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly even foxes kindle blithely even realm" }
+{ "l_orderkey": 2209, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36920.8, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-01", "l_commitdate": "1992-09-25", "l_receiptdate": "1992-11-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ully special sheaves serve" }
 { "l_orderkey": 2496, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35997.78, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-23", "l_commitdate": "1994-02-18", "l_receiptdate": "1994-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "arefully special dependencies abo" }
+{ "l_orderkey": 2566, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16614.36, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-16", "l_commitdate": "1992-12-24", "l_receiptdate": "1992-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " braids according t" }
 { "l_orderkey": 2598, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17537.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-09", "l_commitdate": "1996-05-30", "l_receiptdate": "1996-04-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "nic packages. even accounts" }
-{ "l_orderkey": 2789, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37843.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "d packages-- fluffily specia" }
-{ "l_orderkey": 4896, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eposits hang carefully. sly" }
-{ "l_orderkey": 5347, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31382.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " pending deposits. fluffily regular senti" }
-{ "l_orderkey": 708, "l_partkey": 23, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6461.14, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-16", "l_commitdate": "1998-08-15", "l_receiptdate": "1998-09-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lly express ac" }
-{ "l_orderkey": 1382, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-02", "l_commitdate": "1993-09-29", "l_receiptdate": "1993-10-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ter the carefully final excuses. blit" }
+{ "l_orderkey": 2758, "l_partkey": 23, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15691.34, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-25", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-10-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " accounts! qui" }
 { "l_orderkey": 2759, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 31.0, "l_extendedprice": 28613.62, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-11", "l_commitdate": "1994-01-15", "l_receiptdate": "1994-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ithely aft" }
 { "l_orderkey": 2786, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 22152.48, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-04", "l_commitdate": "1992-06-09", "l_receiptdate": "1992-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ans. slyly unusual platelets detect. unus" }
+{ "l_orderkey": 2789, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37843.82, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-05-15", "l_receiptdate": "1998-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "d packages-- fluffily specia" }
+{ "l_orderkey": 4609, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42458.92, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-11", "l_commitdate": "1997-01-16", "l_receiptdate": "1997-03-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "r foxes. fluffily ironic ideas ha" }
 { "l_orderkey": 4640, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 33228.72, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-03", "l_commitdate": "1996-03-09", "l_receiptdate": "1996-01-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "iously furious accounts boost. carefully" }
 { "l_orderkey": 4834, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31382.68, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-09", "l_commitdate": "1996-11-26", "l_receiptdate": "1996-12-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ounts haggle bo" }
+{ "l_orderkey": 4896, "l_partkey": 23, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4615.1, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-02", "l_commitdate": "1992-11-11", "l_receiptdate": "1992-12-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eposits hang carefully. sly" }
+{ "l_orderkey": 5347, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31382.68, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-18", "l_commitdate": "1995-04-04", "l_receiptdate": "1995-06-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " pending deposits. fluffily regular senti" }
+{ "l_orderkey": 5987, "l_partkey": 23, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 923.02, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-29", "l_receiptdate": "1996-09-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "refully final excuses haggle furiously ag" }
 { "l_orderkey": 67, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3688.08, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-17", "l_commitdate": "1997-01-31", "l_receiptdate": "1997-04-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " cajole thinly expres" }
-{ "l_orderkey": 966, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 38724.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sly ironic asymptotes hagg" }
-{ "l_orderkey": 1348, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37802.82, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. platelets about the ca" }
-{ "l_orderkey": 3362, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12908.28, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "even Tires" }
-{ "l_orderkey": 4678, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21206.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-03", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ily sly deposi" }
-{ "l_orderkey": 4803, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-02-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "t blithely slyly special decoys. " }
-{ "l_orderkey": 5090, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-05-25", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ets integrate ironic, regul" }
 { "l_orderkey": 901, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33192.72, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-11", "l_commitdate": "1998-10-09", "l_receiptdate": "1998-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ". accounts are care" }
+{ "l_orderkey": 966, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 38724.84, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-15", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-05", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sly ironic asymptotes hagg" }
 { "l_orderkey": 1090, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4610.1, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1997-12-25", "l_receiptdate": "1998-02-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "s above the " }
 { "l_orderkey": 1285, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11064.24, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-21", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ss foxes. blithe theodolites cajole slyly" }
-{ "l_orderkey": 1637, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9220.2, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-03-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uriously? blithely even sauternes wake. " }
-{ "l_orderkey": 2566, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8298.18, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-12-28", "l_receiptdate": "1992-12-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "blithely bold accounts? quickl" }
-{ "l_orderkey": 4997, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42412.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ecial courts are carefully" }
+{ "l_orderkey": 1348, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37802.82, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-02", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "kages. platelets about the ca" }
 { "l_orderkey": 1447, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7376.16, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1993-01-12", "l_receiptdate": "1992-12-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ost carefully " }
 { "l_orderkey": 1510, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 46101.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-01", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-11-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "even packages. carefully regular fo" }
+{ "l_orderkey": 1637, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9220.2, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-21", "l_commitdate": "1995-03-17", "l_receiptdate": "1995-03-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "uriously? blithely even sauternes wake. " }
 { "l_orderkey": 1667, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26738.58, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-15", "l_commitdate": "1997-11-09", "l_receiptdate": "1997-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "l accounts. furiously final courts h" }
-{ "l_orderkey": 3073, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eposits. fluffily" }
-{ "l_orderkey": 3492, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1995-01-18", "l_receiptdate": "1994-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ronic instructions u" }
-{ "l_orderkey": 3719, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32270.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly foxes. pending braids haggle furio" }
-{ "l_orderkey": 3974, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-08", "l_receiptdate": "1996-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "dencies above the re" }
-{ "l_orderkey": 4551, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-12", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ly ironic reques" }
-{ "l_orderkey": 5572, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests cajole. evenly ironic exc" }
+{ "l_orderkey": 2566, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8298.18, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-14", "l_commitdate": "1992-12-28", "l_receiptdate": "1992-12-16", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "blithely bold accounts? quickl" }
 { "l_orderkey": 2626, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41490.9, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-22", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-11-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "deposits wake blithely according to " }
 { "l_orderkey": 2757, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11064.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-01", "l_commitdate": "1995-09-04", "l_receiptdate": "1995-08-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, eve" }
 { "l_orderkey": 2913, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-08-11", "l_receiptdate": "1997-10-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "riously pending realms. blithely even pac" }
+{ "l_orderkey": 3073, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-02-12", "l_receiptdate": "1994-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eposits. fluffily" }
+{ "l_orderkey": 3362, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12908.28, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-01", "l_commitdate": "1995-09-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "even Tires" }
+{ "l_orderkey": 3492, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1995-01-18", "l_receiptdate": "1994-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ronic instructions u" }
 { "l_orderkey": 3553, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16596.36, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-03", "l_commitdate": "1994-06-30", "l_receiptdate": "1994-07-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": ". quickly ironic" }
+{ "l_orderkey": 3719, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 32270.7, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-11", "l_commitdate": "1997-04-03", "l_receiptdate": "1997-06-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly foxes. pending braids haggle furio" }
 { "l_orderkey": 3970, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 34.0, "l_extendedprice": 31348.68, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-25", "l_commitdate": "1992-05-23", "l_receiptdate": "1992-07-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "y final gifts are. carefully pe" }
+{ "l_orderkey": 3974, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 43334.94, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-08", "l_receiptdate": "1996-06-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "dencies above the re" }
 { "l_orderkey": 4418, "l_partkey": 22, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12908.28, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-18", "l_receiptdate": "1993-06-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely regular requests. blith" }
-{ "l_orderkey": 67, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11052.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " even packages cajole" }
-{ "l_orderkey": 481, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17499.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p blithely after t" }
-{ "l_orderkey": 1698, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20262.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "oward the furiously iro" }
-{ "l_orderkey": 1921, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "to beans. even excuses integrate specia" }
-{ "l_orderkey": 2917, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18420.4, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-01-22", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly even ideas wa" }
-{ "l_orderkey": 3718, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36840.8, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "out the express deposits" }
-{ "l_orderkey": 3841, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28551.62, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "n theodolites shall promise carefully. qui" }
-{ "l_orderkey": 4327, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7368.16, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites cajole; unusual Tiresias" }
-{ "l_orderkey": 1024, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 45129.98, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-10", "l_receiptdate": "1998-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully bold " }
-{ "l_orderkey": 1667, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5526.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-11-16", "l_receiptdate": "1998-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "riously busy requests. blithely final a" }
-{ "l_orderkey": 1827, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6447.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "egular foxes" }
-{ "l_orderkey": 1985, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 46051.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ate carefully. carefully" }
-{ "l_orderkey": 2468, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 39603.86, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-08-21", "l_receiptdate": "1997-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously eve" }
+{ "l_orderkey": 4551, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-12", "l_commitdate": "1996-03-17", "l_receiptdate": "1996-05-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ly ironic reques" }
+{ "l_orderkey": 4678, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21206.46, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-03", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-09-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ily sly deposi" }
+{ "l_orderkey": 4803, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-24", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-02-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "t blithely slyly special decoys. " }
+{ "l_orderkey": 4997, "l_partkey": 22, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 42412.92, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-06-04", "l_receiptdate": "1998-05-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ecial courts are carefully" }
+{ "l_orderkey": 5090, "l_partkey": 22, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20284.44, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-10", "l_commitdate": "1997-05-25", "l_receiptdate": "1997-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ets integrate ironic, regul" }
+{ "l_orderkey": 5572, "l_partkey": 22, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 22128.48, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-30", "l_commitdate": "1994-10-02", "l_receiptdate": "1994-11-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ests cajole. evenly ironic exc" }
 { "l_orderkey": 39, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29472.64, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-12-19", "l_receiptdate": "1996-10-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "heodolites sleep silently pending foxes. ac" }
-{ "l_orderkey": 549, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16578.36, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-09-11", "l_receiptdate": "1992-08-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ely regular accounts above the " }
-{ "l_orderkey": 2530, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly ironic" }
-{ "l_orderkey": 2949, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3684.08, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "gular pinto beans wake alongside of the reg" }
-{ "l_orderkey": 4581, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42366.92, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nag toward the carefully final accounts. " }
+{ "l_orderkey": 67, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 11052.24, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-27", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-22", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " even packages cajole" }
 { "l_orderkey": 160, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31314.68, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-03-13", "l_receiptdate": "1997-02-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "st sleep even gifts. dependencies along" }
+{ "l_orderkey": 481, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17499.38, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "p blithely after t" }
+{ "l_orderkey": 549, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 18.0, "l_extendedprice": 16578.36, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-31", "l_commitdate": "1992-09-11", "l_receiptdate": "1992-08-08", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ely regular accounts above the " }
+{ "l_orderkey": 1024, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 45129.98, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-10", "l_receiptdate": "1998-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " carefully bold " }
 { "l_orderkey": 1287, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23946.52, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-09-27", "l_receiptdate": "1994-10-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egular foxes. theodolites nag along t" }
 { "l_orderkey": 1604, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21183.46, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-11", "l_commitdate": "1993-08-30", "l_receiptdate": "1993-10-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "en requests. blithely fin" }
+{ "l_orderkey": 1667, "l_partkey": 21, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5526.12, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-11-16", "l_receiptdate": "1998-01-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "riously busy requests. blithely final a" }
+{ "l_orderkey": 1698, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 20262.44, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-05-28", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "oward the furiously iro" }
+{ "l_orderkey": 1827, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6447.14, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-28", "l_commitdate": "1996-08-07", "l_receiptdate": "1996-08-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "egular foxes" }
+{ "l_orderkey": 1921, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-03-20", "l_receiptdate": "1994-03-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "to beans. even excuses integrate specia" }
+{ "l_orderkey": 1985, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 46051.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-30", "l_commitdate": "1994-10-18", "l_receiptdate": "1994-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ate carefully. carefully" }
+{ "l_orderkey": 2468, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 39603.86, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-17", "l_commitdate": "1997-08-21", "l_receiptdate": "1997-08-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously eve" }
+{ "l_orderkey": 2530, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8289.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-10", "l_commitdate": "1994-04-30", "l_receiptdate": "1994-05-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "lyly ironic" }
+{ "l_orderkey": 2917, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18420.4, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-31", "l_commitdate": "1998-01-22", "l_receiptdate": "1998-01-12", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "slyly even ideas wa" }
+{ "l_orderkey": 2949, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3684.08, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-17", "l_receiptdate": "1994-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "gular pinto beans wake alongside of the reg" }
 { "l_orderkey": 3303, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13815.3, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-01-22", "l_receiptdate": "1998-02-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " detect sly" }
 { "l_orderkey": 3331, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34998.76, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-24", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-08-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ymptotes haggle across the ca" }
+{ "l_orderkey": 3718, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36840.8, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-20", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "out the express deposits" }
+{ "l_orderkey": 3841, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28551.62, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-24", "l_commitdate": "1994-11-25", "l_receiptdate": "1995-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "n theodolites shall promise carefully. qui" }
 { "l_orderkey": 4064, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 11052.24, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-08", "l_commitdate": "1996-12-18", "l_receiptdate": "1997-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ding to the requests" }
+{ "l_orderkey": 4327, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7368.16, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-05-26", "l_commitdate": "1995-05-28", "l_receiptdate": "1995-06-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites cajole; unusual Tiresias" }
+{ "l_orderkey": 4581, "l_partkey": 21, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 42366.92, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-09", "l_commitdate": "1992-11-27", "l_receiptdate": "1992-09-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "nag toward the carefully final accounts. " }
 { "l_orderkey": 4642, "l_partkey": 21, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 9210.2, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-16", "l_commitdate": "1995-04-28", "l_receiptdate": "1995-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "urts. even deposits nag beneath " }
 { "l_orderkey": 5573, "l_partkey": 21, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29472.64, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-30", "l_commitdate": "1996-10-25", "l_receiptdate": "1996-10-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "egular depths haggl" }
+{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+{ "l_orderkey": 326, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34960.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "es sleep slyly. carefully regular inst" }
+{ "l_orderkey": 865, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2760.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-14", "l_receiptdate": "1993-08-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fully regular the" }
+{ "l_orderkey": 1120, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-03", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "heodolites. quick re" }
+{ "l_orderkey": 1127, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26680.58, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. blithely r" }
+{ "l_orderkey": 1444, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 32200.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle furiou" }
+{ "l_orderkey": 1504, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-23", "l_receiptdate": "1992-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final packa" }
+{ "l_orderkey": 1509, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10120.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-04", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ily ironic packages nod carefully." }
 { "l_orderkey": 1985, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1840.04, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-25", "l_commitdate": "1994-10-09", "l_receiptdate": "1994-12-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " silent inst" }
 { "l_orderkey": 2023, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20240.44, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-15", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-06-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ronic attainments. " }
 { "l_orderkey": 2273, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-19", "l_commitdate": "1997-01-22", "l_receiptdate": "1997-02-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. furiou" }
 { "l_orderkey": 2372, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 5.0, "l_extendedprice": 4600.1, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-08", "l_commitdate": "1998-01-18", "l_receiptdate": "1998-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ets against the " }
+{ "l_orderkey": 2625, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38640.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-10-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even accounts haggle furiously" }
 { "l_orderkey": 2694, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 11040.24, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-24", "l_commitdate": "1996-04-22", "l_receiptdate": "1996-05-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "foxes atop the hockey pla" }
 { "l_orderkey": 3202, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 20240.44, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-16", "l_commitdate": "1993-02-16", "l_receiptdate": "1993-03-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the express packages. fu" }
+{ "l_orderkey": 3330, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "haggle carefully alongside of the bold r" }
+{ "l_orderkey": 3778, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23920.52, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " against the fluffily" }
+{ "l_orderkey": 3910, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5520.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly sly platelets are fluffily slyly si" }
 { "l_orderkey": 4193, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 50.0, "l_extendedprice": 46001.0, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-28", "l_commitdate": "1994-03-23", "l_receiptdate": "1994-05-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " beans. regular accounts cajole. de" }
+{ "l_orderkey": 4386, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 16.0, "l_extendedprice": 14720.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously final pint" }
+{ "l_orderkey": 4773, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly pending theodolites cajole caref" }
+{ "l_orderkey": 5254, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 8280.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " wake blithely fluff" }
 { "l_orderkey": 5348, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1997-12-20", "l_receiptdate": "1998-02-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "even foxes. epitap" }
 { "l_orderkey": 5444, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 21.0, "l_extendedprice": 19320.42, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-30", "l_commitdate": "1995-05-01", "l_receiptdate": "1995-03-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "furiously even theodolites." }
 { "l_orderkey": 5476, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15640.34, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-02", "l_commitdate": "1998-01-28", "l_receiptdate": "1998-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ng dependencies until the f" }
 { "l_orderkey": 5510, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42320.92, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-12", "l_commitdate": "1993-02-09", "l_receiptdate": "1993-03-19", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "silent packages cajole doggedly regular " }
-{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
-{ "l_orderkey": 326, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34960.76, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-12", "l_commitdate": "1995-08-23", "l_receiptdate": "1995-09-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "es sleep slyly. carefully regular inst" }
-{ "l_orderkey": 865, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2760.06, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-14", "l_receiptdate": "1993-08-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fully regular the" }
-{ "l_orderkey": 1127, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26680.58, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-20", "l_commitdate": "1995-11-21", "l_receiptdate": "1995-10-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y. blithely r" }
-{ "l_orderkey": 1509, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10120.22, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-04", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ily ironic packages nod carefully." }
-{ "l_orderkey": 3330, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-02", "l_commitdate": "1995-03-03", "l_receiptdate": "1995-03-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "haggle carefully alongside of the bold r" }
-{ "l_orderkey": 5254, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 8280.18, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-10-15", "l_receiptdate": "1992-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " wake blithely fluff" }
-{ "l_orderkey": 1120, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-03", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-01-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "heodolites. quick re" }
-{ "l_orderkey": 1444, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 32200.7, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-25", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "aggle furiou" }
-{ "l_orderkey": 1504, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6440.14, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-23", "l_receiptdate": "1992-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "y final packa" }
-{ "l_orderkey": 2625, "l_partkey": 20, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38640.84, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-18", "l_commitdate": "1992-11-17", "l_receiptdate": "1992-10-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " even accounts haggle furiously" }
-{ "l_orderkey": 3778, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23920.52, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " against the fluffily" }
-{ "l_orderkey": 3910, "l_partkey": 20, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5520.12, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-30", "l_receiptdate": "1996-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ly sly platelets are fluffily slyly si" }
-{ "l_orderkey": 4386, "l_partkey": 20, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 16.0, "l_extendedprice": 14720.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-05", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously final pint" }
-{ "l_orderkey": 4773, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 45080.98, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1996-02-17", "l_receiptdate": "1996-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly pending theodolites cajole caref" }
 { "l_orderkey": 5956, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36800.8, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-11", "l_commitdate": "1998-07-19", "l_receiptdate": "1998-06-21", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "final theodolites sleep carefully ironic c" }
+{ "l_orderkey": 69, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 21137.23, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-10-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding accounts ca" }
 { "l_orderkey": 164, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 22056.24, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-22", "l_commitdate": "1992-11-27", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "side of the slyly unusual theodolites. f" }
 { "l_orderkey": 230, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7352.08, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-03", "l_commitdate": "1994-01-20", "l_receiptdate": "1993-11-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "g the instructions. fluffil" }
-{ "l_orderkey": 418, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28489.31, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "final theodolites. fluffil" }
-{ "l_orderkey": 1634, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19299.21, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-11-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y along the excuses." }
-{ "l_orderkey": 2304, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits cajole blithely e" }
-{ "l_orderkey": 3585, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36760.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-22", "l_commitdate": "1995-01-17", "l_receiptdate": "1995-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets affix. even asymptotes play care" }
-{ "l_orderkey": 3651, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tect quickly among the r" }
-{ "l_orderkey": 3874, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-06-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ideas throughout " }
-{ "l_orderkey": 4263, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uietly regular deposits. sly deposits w" }
-{ "l_orderkey": 481, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". quickly final accounts among the " }
-{ "l_orderkey": 4962, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42274.46, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " pinto beans grow about the sl" }
-{ "l_orderkey": 69, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 23.0, "l_extendedprice": 21137.23, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-03", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-10-24", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "nding accounts ca" }
 { "l_orderkey": 325, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 32165.35, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-06", "l_commitdate": "1994-01-03", "l_receiptdate": "1993-12-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "packages wa" }
+{ "l_orderkey": 418, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28489.31, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-05", "l_commitdate": "1995-06-18", "l_receiptdate": "1995-06-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "final theodolites. fluffil" }
+{ "l_orderkey": 481, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-11-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": ". quickly final accounts among the " }
 { "l_orderkey": 999, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2757.03, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-17", "l_commitdate": "1993-10-22", "l_receiptdate": "1993-10-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "nic, pending ideas. bl" }
-{ "l_orderkey": 2149, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11028.12, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "riously bl" }
-{ "l_orderkey": 2437, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9190.1, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-23", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts. even, ironic pl" }
-{ "l_orderkey": 2503, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-31", "l_receiptdate": "1993-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts haggle blithel" }
-{ "l_orderkey": 2694, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13785.15, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e blithely even platelets. special wa" }
-{ "l_orderkey": 3207, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29408.32, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the slyly express foxes. bl" }
-{ "l_orderkey": 3425, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34003.37, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-10", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ngside of the furiously thin dol" }
-{ "l_orderkey": 5127, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30327.33, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold deposits use carefully a" }
-{ "l_orderkey": 5344, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5514.06, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ithely about the pending plate" }
+{ "l_orderkey": 1634, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19299.21, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-16", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-11-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y along the excuses." }
 { "l_orderkey": 1636, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20218.22, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-22", "l_commitdate": "1997-08-18", "l_receiptdate": "1997-08-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ular, regu" }
 { "l_orderkey": 1888, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8271.09, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-02-19", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " packages are blithely. carefu" }
 { "l_orderkey": 2023, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22975.25, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-19", "l_commitdate": "1992-07-07", "l_receiptdate": "1992-08-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " wake furiously among the slyly final" }
+{ "l_orderkey": 2149, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 11028.12, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-01", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "riously bl" }
+{ "l_orderkey": 2304, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-12", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-03-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " deposits cajole blithely e" }
+{ "l_orderkey": 2437, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9190.1, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-06-23", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "unts. even, ironic pl" }
+{ "l_orderkey": 2503, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 17.0, "l_extendedprice": 15623.17, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-04", "l_commitdate": "1993-07-31", "l_receiptdate": "1993-09-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "c accounts haggle blithel" }
+{ "l_orderkey": 2694, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13785.15, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-30", "l_commitdate": "1996-05-01", "l_receiptdate": "1996-07-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "e blithely even platelets. special wa" }
 { "l_orderkey": 2695, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40436.44, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-10", "l_receiptdate": "1996-11-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ts. busy platelets boost" }
 { "l_orderkey": 3044, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 43193.47, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-24", "l_commitdate": "1996-06-22", "l_receiptdate": "1996-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ly around the car" }
+{ "l_orderkey": 3207, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29408.32, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-17", "l_commitdate": "1998-04-26", "l_receiptdate": "1998-07-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "y across the slyly express foxes. bl" }
+{ "l_orderkey": 3425, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 37.0, "l_extendedprice": 34003.37, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-10", "l_commitdate": "1996-05-10", "l_receiptdate": "1996-08-02", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ngside of the furiously thin dol" }
+{ "l_orderkey": 3585, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36760.4, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-22", "l_commitdate": "1995-01-17", "l_receiptdate": "1995-02-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "elets affix. even asymptotes play care" }
+{ "l_orderkey": 3651, "l_partkey": 19, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-10", "l_commitdate": "1998-06-06", "l_receiptdate": "1998-06-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tect quickly among the r" }
 { "l_orderkey": 3719, "l_partkey": 19, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 16.0, "l_extendedprice": 14704.16, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-02", "l_commitdate": "1997-03-18", "l_receiptdate": "1997-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " express asymptotes. ir" }
+{ "l_orderkey": 3874, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 44112.48, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-13", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-06-20", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ideas throughout " }
+{ "l_orderkey": 4263, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18380.2, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-09", "l_commitdate": "1998-04-30", "l_receiptdate": "1998-05-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uietly regular deposits. sly deposits w" }
+{ "l_orderkey": 4962, "l_partkey": 19, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 42274.46, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-23", "l_commitdate": "1993-09-04", "l_receiptdate": "1993-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " pinto beans grow about the sl" }
+{ "l_orderkey": 5127, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30327.33, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-03-02", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " bold deposits use carefully a" }
+{ "l_orderkey": 5344, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5514.06, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-04", "l_commitdate": "1998-09-03", "l_receiptdate": "1998-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ithely about the pending plate" }
 { "l_orderkey": 5538, "l_partkey": 19, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34922.38, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-17", "l_commitdate": "1994-02-11", "l_receiptdate": "1994-04-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ular pinto beans. silent ideas above " }
+{ "l_orderkey": 164, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts wake again" }
 { "l_orderkey": 197, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22950.25, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-13", "l_commitdate": "1995-05-23", "l_receiptdate": "1995-06-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "s-- quickly final accounts" }
 { "l_orderkey": 768, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1836.02, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-13", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-11-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ular courts. slyly dogged accou" }
+{ "l_orderkey": 901, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10098.11, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-13", "l_commitdate": "1998-10-19", "l_receiptdate": "1998-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ourts among the quickly expre" }
 { "l_orderkey": 930, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-04-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ackages. fluffily e" }
+{ "l_orderkey": 965, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21114.23, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ld kindle carefully across th" }
+{ "l_orderkey": 1537, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he regular pack" }
+{ "l_orderkey": 1920, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5508.06, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-01", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-10-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l ideas boost slyly pl" }
+{ "l_orderkey": 1924, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-18", "l_receiptdate": "1996-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "silent requests cajole blithely final pack" }
+{ "l_orderkey": 1955, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1836.02, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ickly aroun" }
+{ "l_orderkey": 2150, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26622.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-02", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic theodolites. foxes ca" }
 { "l_orderkey": 2151, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25704.28, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-12-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y special packages. carefully ironic instru" }
 { "l_orderkey": 2246, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 10098.11, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-21", "l_commitdate": "1996-07-24", "l_receiptdate": "1996-07-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "quests alongside o" }
+{ "l_orderkey": 2404, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 37638.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-07-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " dolphins are" }
 { "l_orderkey": 2658, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11934.13, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-12-12", "l_receiptdate": "1995-11-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s kindle blithely regular accounts." }
 { "l_orderkey": 2688, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 41310.45, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-21", "l_commitdate": "1992-04-14", "l_receiptdate": "1992-05-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sits run carefully" }
 { "l_orderkey": 2818, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38556.42, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-19", "l_receiptdate": "1995-03-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ar accounts wake carefully a" }
-{ "l_orderkey": 3109, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ecial orbits are furiou" }
-{ "l_orderkey": 4263, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8262.09, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-04-29", "l_receiptdate": "1998-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "structions cajole quic" }
-{ "l_orderkey": 5186, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25704.28, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "al decoys. blit" }
-{ "l_orderkey": 901, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10098.11, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-13", "l_commitdate": "1998-10-19", "l_receiptdate": "1998-11-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ourts among the quickly expre" }
-{ "l_orderkey": 1955, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1836.02, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-06", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ickly aroun" }
-{ "l_orderkey": 2404, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 37638.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-12", "l_commitdate": "1997-05-03", "l_receiptdate": "1997-07-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " dolphins are" }
 { "l_orderkey": 2854, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 13.0, "l_extendedprice": 11934.13, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-15", "l_commitdate": "1994-08-18", "l_receiptdate": "1994-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " excuses wak" }
-{ "l_orderkey": 3777, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32130.35, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s. carefully express asymptotes accordi" }
-{ "l_orderkey": 965, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21114.23, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-12", "l_commitdate": "1995-07-08", "l_receiptdate": "1995-08-11", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ld kindle carefully across th" }
-{ "l_orderkey": 1920, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5508.06, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-01", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-10-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "l ideas boost slyly pl" }
-{ "l_orderkey": 1924, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-24", "l_commitdate": "1996-10-18", "l_receiptdate": "1996-12-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "silent requests cajole blithely final pack" }
-{ "l_orderkey": 2150, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26622.29, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-02", "l_commitdate": "1994-08-04", "l_receiptdate": "1994-10-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y ironic theodolites. foxes ca" }
-{ "l_orderkey": 3584, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 35802.39, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits. carefu" }
-{ "l_orderkey": 4833, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23868.26, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-13", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-05-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s packages. even gif" }
-{ "l_orderkey": 5829, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "after the furiously ironic ideas no" }
-{ "l_orderkey": 164, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-21", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ts wake again" }
-{ "l_orderkey": 1537, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-04-19", "l_receiptdate": "1992-04-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "he regular pack" }
 { "l_orderkey": 2880, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 42228.46, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-06-05", "l_receiptdate": "1992-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eep quickly according to t" }
 { "l_orderkey": 3015, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-11-20", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s above the fluffily final t" }
 { "l_orderkey": 3040, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9180.1, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-16", "l_commitdate": "1993-06-24", "l_receiptdate": "1993-06-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ely regular foxes haggle dari" }
+{ "l_orderkey": 3109, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29376.32, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-05", "l_commitdate": "1993-10-06", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ecial orbits are furiou" }
 { "l_orderkey": 3175, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-09-10", "l_receiptdate": "1994-08-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final requests x-r" }
+{ "l_orderkey": 3584, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 39.0, "l_extendedprice": 35802.39, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-10-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "eposits. carefu" }
+{ "l_orderkey": 3777, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 32130.35, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-25", "l_commitdate": "1994-05-26", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "s. carefully express asymptotes accordi" }
 { "l_orderkey": 4071, "l_partkey": 18, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43146.47, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ts cajole furiously along the" }
+{ "l_orderkey": 4263, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8262.09, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-04", "l_commitdate": "1998-04-29", "l_receiptdate": "1998-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "structions cajole quic" }
+{ "l_orderkey": 4833, "l_partkey": 18, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23868.26, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-13", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-05-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "s packages. even gif" }
 { "l_orderkey": 4960, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 33048.36, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-06", "l_commitdate": "1995-05-04", "l_receiptdate": "1995-04-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c, unusual accou" }
+{ "l_orderkey": 5186, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25704.28, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-05", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "al decoys. blit" }
 { "l_orderkey": 5699, "l_partkey": 18, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 44064.48, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-23", "l_commitdate": "1992-10-20", "l_receiptdate": "1992-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s. carefully regul" }
+{ "l_orderkey": 5829, "l_partkey": 18, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 17.0, "l_extendedprice": 15606.17, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-29", "l_receiptdate": "1997-04-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "after the furiously ironic ideas no" }
+{ "l_orderkey": 289, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40348.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic foxes. asymptotes " }
+{ "l_orderkey": 611, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35763.39, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nto beans " }
+{ "l_orderkey": 647, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-09-24", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r instructions. quickly unusu" }
 { "l_orderkey": 774, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-13", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-03-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " slyly even courts nag blith" }
 { "l_orderkey": 931, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9170.1, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-01", "l_commitdate": "1993-01-09", "l_receiptdate": "1993-03-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ajole quickly. slyly sil" }
 { "l_orderkey": 967, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-23", "l_commitdate": "1992-08-07", "l_receiptdate": "1992-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "efully special ide" }
-{ "l_orderkey": 2692, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests. bold, even foxes haggle slyl" }
-{ "l_orderkey": 2944, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21091.23, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-03", "l_receiptdate": "1998-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " excuses? regular platelets e" }
-{ "l_orderkey": 3621, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-08-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "al requests. fl" }
-{ "l_orderkey": 3872, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34846.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously against the ironic, unusual a" }
-{ "l_orderkey": 4005, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25676.28, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-01-24", "l_receiptdate": "1996-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly carefully ironic deposits. slyly" }
-{ "l_orderkey": 289, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40348.44, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-04-20", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly ironic foxes. asymptotes " }
 { "l_orderkey": 1411, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8253.09, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-08", "l_commitdate": "1995-03-04", "l_receiptdate": "1995-03-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "accounts. furiou" }
 { "l_orderkey": 1825, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6419.07, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-02", "l_commitdate": "1994-01-30", "l_receiptdate": "1994-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "fully ironic requests. requests cajole ex" }
-{ "l_orderkey": 2437, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress dolphins. furiously fin" }
-{ "l_orderkey": 2565, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22925.25, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ", express accounts. final id" }
-{ "l_orderkey": 2720, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 38514.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fter the inst" }
-{ "l_orderkey": 3079, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36680.4, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-12-11", "l_receiptdate": "1997-10-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ide of the pending, special deposi" }
-{ "l_orderkey": 4673, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7336.08, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely final re" }
-{ "l_orderkey": 5159, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42182.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s kindle slyly carefully regular" }
-{ "l_orderkey": 5313, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31178.34, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccording to the blithely final account" }
-{ "l_orderkey": 611, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35763.39, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-06", "l_commitdate": "1993-04-09", "l_receiptdate": "1993-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "nto beans " }
 { "l_orderkey": 1958, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 44.0, "l_extendedprice": 40348.44, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-17", "l_commitdate": "1995-11-30", "l_receiptdate": "1996-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "c theodolites after the unusual deposit" }
-{ "l_orderkey": 5189, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-12", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ial theodolites cajole slyly. slyly unus" }
-{ "l_orderkey": 5348, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14672.16, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-28", "l_commitdate": "1997-12-25", "l_receiptdate": "1998-03-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uriously thin pinto beans " }
-{ "l_orderkey": 647, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-19", "l_commitdate": "1997-09-24", "l_receiptdate": "1997-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "r instructions. quickly unusu" }
 { "l_orderkey": 2400, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21091.23, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-10-04", "l_receiptdate": "1998-10-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ages lose carefully around the regula" }
 { "l_orderkey": 2405, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 44933.49, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-03-23", "l_receiptdate": "1997-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "cial requests. ironic, regu" }
+{ "l_orderkey": 2437, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-12", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-05-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ress dolphins. furiously fin" }
+{ "l_orderkey": 2565, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22925.25, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-27", "l_commitdate": "1998-05-20", "l_receiptdate": "1998-07-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ", express accounts. final id" }
+{ "l_orderkey": 2692, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2751.03, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-25", "l_commitdate": "1998-01-29", "l_receiptdate": "1998-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "equests. bold, even foxes haggle slyl" }
+{ "l_orderkey": 2720, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 42.0, "l_extendedprice": 38514.42, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-25", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "fter the inst" }
+{ "l_orderkey": 2944, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 21091.23, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-12-03", "l_receiptdate": "1998-01-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " excuses? regular platelets e" }
+{ "l_orderkey": 3079, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36680.4, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-26", "l_commitdate": "1997-12-11", "l_receiptdate": "1997-10-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ide of the pending, special deposi" }
+{ "l_orderkey": 3621, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26593.29, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-08", "l_receiptdate": "1993-08-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "al requests. fl" }
+{ "l_orderkey": 3872, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34846.38, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-18", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously against the ironic, unusual a" }
 { "l_orderkey": 3943, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 29344.32, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-22", "l_commitdate": "1996-12-17", "l_receiptdate": "1996-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " unusual ideas into the furiously even pack" }
+{ "l_orderkey": 4005, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25676.28, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-11", "l_commitdate": "1997-01-24", "l_receiptdate": "1996-12-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ly carefully ironic deposits. slyly" }
 { "l_orderkey": 4262, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 26.0, "l_extendedprice": 23842.26, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-29", "l_commitdate": "1996-09-25", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s boost slyly along the bold, iro" }
 { "l_orderkey": 4355, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3668.04, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-25", "l_commitdate": "1997-01-29", "l_receiptdate": "1997-03-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "slyly blithely regular packag" }
+{ "l_orderkey": 4673, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7336.08, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-12", "l_commitdate": "1996-10-05", "l_receiptdate": "1996-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "lithely final re" }
 { "l_orderkey": 4997, "l_partkey": 17, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4585.05, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-16", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cuses are furiously unusual asymptotes" }
+{ "l_orderkey": 5159, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42182.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-15", "l_commitdate": "1996-12-07", "l_receiptdate": "1996-12-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s kindle slyly carefully regular" }
+{ "l_orderkey": 5189, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37597.41, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-12", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-01-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ial theodolites cajole slyly. slyly unus" }
+{ "l_orderkey": 5313, "l_partkey": 17, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31178.34, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-08-12", "l_receiptdate": "1997-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ccording to the blithely final account" }
+{ "l_orderkey": 5348, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 14672.16, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-28", "l_commitdate": "1997-12-25", "l_receiptdate": "1998-03-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "uriously thin pinto beans " }
 { "l_orderkey": 5827, "l_partkey": 17, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 14.0, "l_extendedprice": 12838.14, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-31", "l_commitdate": "1998-09-06", "l_receiptdate": "1998-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rges. fluffily pending " }
 { "l_orderkey": 5924, "l_partkey": 17, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 22008.24, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-12", "l_commitdate": "1995-12-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " use carefully. special, e" }
+{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+{ "l_orderkey": 198, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18320.2, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully final escapades a" }
+{ "l_orderkey": 295, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-11-17", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final instructions h" }
 { "l_orderkey": 420, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 35724.39, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-09", "l_commitdate": "1995-12-16", "l_receiptdate": "1995-12-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "s. ironic waters about the car" }
+{ "l_orderkey": 1315, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13740.15, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". foxes integrate carefully special" }
+{ "l_orderkey": 1346, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 41220.45, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "press deposits." }
+{ "l_orderkey": 1696, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-05-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the blithely" }
+{ "l_orderkey": 1861, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1832.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e final, regular requests. carefully " }
+{ "l_orderkey": 1987, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6412.07, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular a" }
+{ "l_orderkey": 2178, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24732.27, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across the ironic reques" }
 { "l_orderkey": 2180, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28396.31, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-20", "l_commitdate": "1996-11-21", "l_receiptdate": "1996-11-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "n requests are furiously at the quickly" }
 { "l_orderkey": 2789, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 47.0, "l_extendedprice": 43052.47, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-04-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "usly busy packages wake against the unusual" }
-{ "l_orderkey": 4771, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19236.21, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-19", "l_commitdate": "1993-02-10", "l_receiptdate": "1993-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fluffily pendi" }
-{ "l_orderkey": 5286, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2748.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re fluffily" }
-{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
-{ "l_orderkey": 295, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-11-17", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " final instructions h" }
-{ "l_orderkey": 1315, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13740.15, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-06-10", "l_receiptdate": "1998-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". foxes integrate carefully special" }
-{ "l_orderkey": 2178, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24732.27, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-26", "l_commitdate": "1997-02-19", "l_receiptdate": "1997-03-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " across the ironic reques" }
-{ "l_orderkey": 3781, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21068.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "pendencies are b" }
-{ "l_orderkey": 5184, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43052.47, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "se. carefully express pinto beans x" }
-{ "l_orderkey": 5189, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34808.38, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ideas. idle, final deposits de" }
-{ "l_orderkey": 5415, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39388.43, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-09-14", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "yly blithely stealthy deposits. carefu" }
-{ "l_orderkey": 5442, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22900.25, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ake furiously. slyly express th" }
-{ "l_orderkey": 5510, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-16", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "n packages boost sly" }
-{ "l_orderkey": 5858, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 32976.36, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "osits wake quickly quickly sile" }
-{ "l_orderkey": 198, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 18320.2, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-15", "l_commitdate": "1998-03-31", "l_receiptdate": "1998-01-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "carefully final escapades a" }
-{ "l_orderkey": 1987, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6412.07, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-06", "l_receiptdate": "1994-08-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " regular a" }
 { "l_orderkey": 3040, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16488.18, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-07-06", "l_receiptdate": "1993-07-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ly thin accou" }
 { "l_orderkey": 3365, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1832.02, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-04", "l_commitdate": "1994-12-30", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "es cajole fluffily pe" }
 { "l_orderkey": 3458, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14656.16, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-01", "l_commitdate": "1995-02-25", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "s grow carefully. express, final grouc" }
 { "l_orderkey": 3717, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 6412.07, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-08", "l_commitdate": "1998-07-18", "l_receiptdate": "1998-09-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": " after the packa" }
+{ "l_orderkey": 3781, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 21068.23, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-05", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-09-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "pendencies are b" }
+{ "l_orderkey": 4771, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 19236.21, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-19", "l_commitdate": "1993-02-10", "l_receiptdate": "1993-02-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "fluffily pendi" }
 { "l_orderkey": 4806, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23816.26, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-28", "l_commitdate": "1993-06-07", "l_receiptdate": "1993-05-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " bold pearls sublate blithely. quickly pe" }
 { "l_orderkey": 4966, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 23816.26, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-11-29", "l_receiptdate": "1996-12-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "nt pearls haggle carefully slyly even " }
-{ "l_orderkey": 1346, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 45.0, "l_extendedprice": 41220.45, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-11", "l_commitdate": "1992-08-06", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "press deposits." }
-{ "l_orderkey": 1696, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-28", "l_commitdate": "1998-02-07", "l_receiptdate": "1998-05-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "the blithely" }
-{ "l_orderkey": 1861, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 2.0, "l_extendedprice": 1832.02, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-26", "l_commitdate": "1994-03-15", "l_receiptdate": "1994-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e final, regular requests. carefully " }
+{ "l_orderkey": 5184, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 43052.47, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-11-02", "l_commitdate": "1998-08-19", "l_receiptdate": "1998-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "se. carefully express pinto beans x" }
+{ "l_orderkey": 5189, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34808.38, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-26", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ideas. idle, final deposits de" }
 { "l_orderkey": 5217, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21068.23, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-18", "l_commitdate": "1995-12-24", "l_receiptdate": "1996-02-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ven ideas. requests amo" }
+{ "l_orderkey": 5286, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2748.03, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1997-11-06", "l_receiptdate": "1997-12-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "re fluffily" }
+{ "l_orderkey": 5415, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 43.0, "l_extendedprice": 39388.43, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-17", "l_commitdate": "1992-09-14", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "yly blithely stealthy deposits. carefu" }
+{ "l_orderkey": 5442, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22900.25, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-02-13", "l_receiptdate": "1998-04-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "ake furiously. slyly express th" }
+{ "l_orderkey": 5510, "l_partkey": 16, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7328.08, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-03-16", "l_commitdate": "1993-03-29", "l_receiptdate": "1993-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "n packages boost sly" }
 { "l_orderkey": 5697, "l_partkey": 16, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 39388.43, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-08", "l_commitdate": "1992-12-03", "l_receiptdate": "1992-12-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "blithely reg" }
+{ "l_orderkey": 5858, "l_partkey": 16, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 32976.36, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-08-16", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "osits wake quickly quickly sile" }
+{ "l_orderkey": 160, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32940.36, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-11", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "old, ironic deposits are quickly abov" }
+{ "l_orderkey": 774, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7320.08, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-15", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ully ironic requests c" }
 { "l_orderkey": 1474, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4575.05, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-22", "l_commitdate": "1995-02-20", "l_receiptdate": "1995-05-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ully final a" }
+{ "l_orderkey": 2151, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26535.29, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " bold packages acro" }
+{ "l_orderkey": 2309, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4575.05, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-29", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. requests wake blithely specia" }
 { "l_orderkey": 2563, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38430.42, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ymptotes nag furiously slyly even inst" }
+{ "l_orderkey": 2688, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42090.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "elets. regular reque" }
+{ "l_orderkey": 2913, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 11895.13, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inos are carefully alongside of the bol" }
+{ "l_orderkey": 3109, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9150.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits haggle carefully. regular, unusual ac" }
+{ "l_orderkey": 3843, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6405.07, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly even instructions. furiously eve" }
+{ "l_orderkey": 3846, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-03-22", "l_receiptdate": "1998-02-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "efully even packages against the blithe" }
+{ "l_orderkey": 4005, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tions sleep across the silent d" }
 { "l_orderkey": 4065, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42090.46, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-29", "l_commitdate": "1994-08-01", "l_receiptdate": "1994-07-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ", regular requests may mold above the " }
 { "l_orderkey": 4132, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21045.23, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-27", "l_commitdate": "1995-07-27", "l_receiptdate": "1995-07-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "d deposits. fluffily even requests haggle b" }
 { "l_orderkey": 4354, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27450.3, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1994-11-24", "l_receiptdate": "1995-02-25", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "around the ir" }
-{ "l_orderkey": 4832, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21045.23, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1998-01-05", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y express depo" }
-{ "l_orderkey": 774, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 7320.08, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1996-01-15", "l_receiptdate": "1996-02-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ully ironic requests c" }
-{ "l_orderkey": 3109, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 10.0, "l_extendedprice": 9150.1, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-26", "l_commitdate": "1993-10-03", "l_receiptdate": "1993-11-09", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits haggle carefully. regular, unusual ac" }
-{ "l_orderkey": 3846, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-14", "l_commitdate": "1998-03-22", "l_receiptdate": "1998-02-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "efully even packages against the blithe" }
-{ "l_orderkey": 5895, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34770.38, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts are furiously. regular, final excuses " }
-{ "l_orderkey": 5957, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33855.37, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ideas use ruthlessly." }
-{ "l_orderkey": 2151, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26535.29, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-04", "l_commitdate": "1996-12-27", "l_receiptdate": "1997-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " bold packages acro" }
-{ "l_orderkey": 2309, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4575.05, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-10", "l_commitdate": "1995-10-29", "l_receiptdate": "1996-01-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s. requests wake blithely specia" }
-{ "l_orderkey": 2913, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 11895.13, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-10-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inos are carefully alongside of the bol" }
-{ "l_orderkey": 3843, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6405.07, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-13", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-02-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "slyly even instructions. furiously eve" }
 { "l_orderkey": 4450, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8235.09, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-13", "l_commitdate": "1997-08-16", "l_receiptdate": "1997-08-15", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "gular requests cajole carefully. regular c" }
-{ "l_orderkey": 160, "l_partkey": 15, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32940.36, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-11", "l_commitdate": "1997-03-11", "l_receiptdate": "1997-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "old, ironic deposits are quickly abov" }
-{ "l_orderkey": 2688, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 42090.46, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-05-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "elets. regular reque" }
-{ "l_orderkey": 4005, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 49.0, "l_extendedprice": 44835.49, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1996-12-24", "l_receiptdate": "1997-03-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tions sleep across the silent d" }
+{ "l_orderkey": 4832, "l_partkey": 15, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 21045.23, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-05", "l_commitdate": "1998-01-05", "l_receiptdate": "1997-12-10", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y express depo" }
 { "l_orderkey": 5472, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 1.0, "l_extendedprice": 915.01, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-04-14", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-04-16", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s use furiou" }
 { "l_orderkey": 5473, "l_partkey": 15, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 30195.33, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-18", "l_commitdate": "1992-06-10", "l_receiptdate": "1992-06-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "efully above the even, " }
+{ "l_orderkey": 5895, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34770.38, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-05", "l_commitdate": "1997-03-06", "l_receiptdate": "1997-05-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ts are furiously. regular, final excuses " }
+{ "l_orderkey": 5957, "l_partkey": 15, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33855.37, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-18", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-05-11", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " ideas use ruthlessly." }
 { "l_orderkey": 453, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34732.38, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-10", "l_commitdate": "1997-07-24", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "sts cajole. furiously un" }
-{ "l_orderkey": 3047, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21022.23, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly ironi" }
-{ "l_orderkey": 3426, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17366.19, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-02", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly special packages oug" }
-{ "l_orderkey": 3781, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 43872.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-22", "l_commitdate": "1996-08-13", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "equests may cajole careful" }
-{ "l_orderkey": 4548, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pecial theodoli" }
-{ "l_orderkey": 4929, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18280.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final pinto beans detect. final," }
 { "l_orderkey": 899, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10054.11, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-03", "l_commitdate": "1998-06-15", "l_receiptdate": "1998-06-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t the ironic" }
 { "l_orderkey": 1510, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 3.0, "l_extendedprice": 2742.03, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-31", "l_commitdate": "1996-12-03", "l_receiptdate": "1996-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "along the slyly regular pin" }
-{ "l_orderkey": 3232, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20108.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-12-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "thely. furio" }
-{ "l_orderkey": 3399, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "se final courts. exc" }
-{ "l_orderkey": 3751, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35646.39, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-16", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "refully according to the iro" }
-{ "l_orderkey": 5028, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13710.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es are quickly final pains. furiously pend" }
-{ "l_orderkey": 5253, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8226.09, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly express deposits use furiou" }
 { "l_orderkey": 1858, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 30162.33, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-28", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-01-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "tect along the slyly final" }
-{ "l_orderkey": 3042, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1994-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "can wake after the enticingly stealthy i" }
-{ "l_orderkey": 4322, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ounts haggle fluffily ideas. pend" }
-{ "l_orderkey": 5731, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5484.06, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rs. quickly regular theo" }
 { "l_orderkey": 1953, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 31990.35, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-03", "l_commitdate": "1994-02-25", "l_receiptdate": "1994-02-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "among the fur" }
 { "l_orderkey": 2020, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27420.3, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-08", "l_commitdate": "1993-08-11", "l_receiptdate": "1993-09-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly about the blithely ironic foxes. bold" }
 { "l_orderkey": 2279, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10968.12, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-10", "l_commitdate": "1993-03-25", "l_receiptdate": "1993-06-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "lets across the excuses nag quickl" }
 { "l_orderkey": 2368, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 29248.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-23", "l_commitdate": "1993-10-07", "l_receiptdate": "1993-09-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gular courts use blithely around the" }
 { "l_orderkey": 2466, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26506.29, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-11", "l_commitdate": "1994-04-27", "l_receiptdate": "1994-07-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ckages. bold requests nag carefully." }
 { "l_orderkey": 2981, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15538.17, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-17", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": ", unusual packages x-ray. furious" }
+{ "l_orderkey": 3042, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1995-02-03", "l_receiptdate": "1994-12-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "can wake after the enticingly stealthy i" }
+{ "l_orderkey": 3047, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 21022.23, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-20", "l_commitdate": "1997-06-14", "l_receiptdate": "1997-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " slyly ironi" }
+{ "l_orderkey": 3232, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20108.22, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-30", "l_commitdate": "1992-12-09", "l_receiptdate": "1992-12-04", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "thely. furio" }
+{ "l_orderkey": 3399, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-12", "l_commitdate": "1995-05-18", "l_receiptdate": "1995-03-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "se final courts. exc" }
 { "l_orderkey": 3425, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7312.08, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-22", "l_commitdate": "1996-06-07", "l_receiptdate": "1996-07-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "iously regular theodolites wake. s" }
+{ "l_orderkey": 3426, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17366.19, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-02", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "slyly special packages oug" }
+{ "l_orderkey": 3751, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35646.39, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-16", "l_commitdate": "1994-07-11", "l_receiptdate": "1994-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "refully according to the iro" }
+{ "l_orderkey": 3781, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 43872.48, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-22", "l_commitdate": "1996-08-13", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "equests may cajole careful" }
+{ "l_orderkey": 4322, "l_partkey": 14, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 34.0, "l_extendedprice": 31076.34, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-27", "l_commitdate": "1998-04-12", "l_receiptdate": "1998-06-16", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ounts haggle fluffily ideas. pend" }
+{ "l_orderkey": 4548, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19194.21, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-11", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-07-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "pecial theodoli" }
+{ "l_orderkey": 4929, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18280.2, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-12", "l_commitdate": "1996-05-23", "l_receiptdate": "1996-03-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " final pinto beans detect. final," }
+{ "l_orderkey": 5028, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13710.15, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-16", "l_receiptdate": "1992-08-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es are quickly final pains. furiously pend" }
+{ "l_orderkey": 5253, "l_partkey": 14, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 9.0, "l_extendedprice": 8226.09, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-06-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "lyly express deposits use furiou" }
+{ "l_orderkey": 5731, "l_partkey": 14, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5484.06, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-07", "l_commitdate": "1997-06-20", "l_receiptdate": "1997-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rs. quickly regular theo" }
 { "l_orderkey": 5792, "l_partkey": 14, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 12796.14, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-28", "l_commitdate": "1993-06-17", "l_receiptdate": "1993-08-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "olites print carefully" }
+{ "l_orderkey": 37, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 39259.43, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously ste" }
 { "l_orderkey": 322, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 20.0, "l_extendedprice": 18260.2, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-26", "l_commitdate": "1992-05-04", "l_receiptdate": "1992-05-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ckly toward " }
-{ "l_orderkey": 933, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24651.27, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ests. express" }
-{ "l_orderkey": 1634, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 31955.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-11-25", "l_receiptdate": "1996-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cies. regular, special de" }
-{ "l_orderkey": 1828, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36520.4, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s use above the quietly fin" }
-{ "l_orderkey": 2432, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-18", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "riously regular packages. p" }
-{ "l_orderkey": 2438, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28303.31, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t. slyly ironic sh" }
-{ "l_orderkey": 2753, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5478.06, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s accounts" }
-{ "l_orderkey": 5313, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15521.17, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uests wake" }
-{ "l_orderkey": 5763, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22825.25, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-08-16", "l_receiptdate": "1998-10-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inal theodolites. even re" }
-{ "l_orderkey": 5953, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31042.34, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-06-12", "l_receiptdate": "1992-06-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hockey players use furiously against th" }
 { "l_orderkey": 514, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5478.06, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-30", "l_commitdate": "1996-06-04", "l_receiptdate": "1996-06-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "as haggle blithely; quickly s" }
 { "l_orderkey": 579, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 28.0, "l_extendedprice": 25564.28, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-05-24", "l_receiptdate": "1998-07-19", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ic ideas until th" }
-{ "l_orderkey": 935, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7304.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-11-02", "l_receiptdate": "1998-02-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cept the quickly regular p" }
-{ "l_orderkey": 1537, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40172.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar courts." }
-{ "l_orderkey": 2723, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 42911.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "furiously r" }
-{ "l_orderkey": 3239, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11869.13, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "r deposits solve fluf" }
-{ "l_orderkey": 5831, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 41998.46, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-01-18", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly final pa" }
-{ "l_orderkey": 37, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 43.0, "l_extendedprice": 39259.43, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-10", "l_commitdate": "1992-07-06", "l_receiptdate": "1992-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "iously ste" }
 { "l_orderkey": 643, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25564.28, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-13", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ly regular requests nag sly" }
+{ "l_orderkey": 933, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24651.27, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-03", "l_commitdate": "1992-10-02", "l_receiptdate": "1992-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ests. express" }
+{ "l_orderkey": 935, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 8.0, "l_extendedprice": 7304.08, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-12", "l_commitdate": "1997-11-02", "l_receiptdate": "1998-02-05", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "cept the quickly regular p" }
 { "l_orderkey": 1379, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21912.24, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-06", "l_commitdate": "1998-07-09", "l_receiptdate": "1998-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ages cajole carefully idly express re" }
-{ "l_orderkey": 3648, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14608.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously stealthy deposits haggle furi" }
-{ "l_orderkey": 5445, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-19", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests. bravely i" }
+{ "l_orderkey": 1537, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 44.0, "l_extendedprice": 40172.44, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-01", "l_commitdate": "1992-03-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lar courts." }
+{ "l_orderkey": 1634, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 35.0, "l_extendedprice": 31955.35, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-25", "l_commitdate": "1996-11-25", "l_receiptdate": "1996-12-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "cies. regular, special de" }
+{ "l_orderkey": 1828, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36520.4, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s use above the quietly fin" }
 { "l_orderkey": 2081, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 29216.32, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-09-26", "l_receiptdate": "1997-10-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e. final, regular dependencies sleep slyly!" }
+{ "l_orderkey": 2432, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-18", "l_commitdate": "1996-09-04", "l_receiptdate": "1996-08-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "riously regular packages. p" }
+{ "l_orderkey": 2438, "l_partkey": 13, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28303.31, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-08-31", "l_receiptdate": "1993-11-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "t. slyly ironic sh" }
+{ "l_orderkey": 2723, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 42911.47, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-05", "l_commitdate": "1995-11-19", "l_receiptdate": "1995-12-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "furiously r" }
+{ "l_orderkey": 2753, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5478.06, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1994-01-28", "l_receiptdate": "1994-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "s accounts" }
+{ "l_orderkey": 3239, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11869.13, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-10", "l_commitdate": "1998-02-19", "l_receiptdate": "1998-02-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "r deposits solve fluf" }
 { "l_orderkey": 3397, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10043.11, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-29", "l_commitdate": "1994-09-18", "l_receiptdate": "1994-08-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "iously careful packages. s" }
+{ "l_orderkey": 3648, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14608.16, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-07-27", "l_commitdate": "1993-08-26", "l_receiptdate": "1993-08-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "uriously stealthy deposits haggle furi" }
 { "l_orderkey": 4674, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19173.21, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ent accounts sublate deposits. instruc" }
 { "l_orderkey": 4965, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22825.25, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1993-12-15", "l_receiptdate": "1994-02-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "wake at the carefully speci" }
+{ "l_orderkey": 5313, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15521.17, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-09-07", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uests wake" }
+{ "l_orderkey": 5445, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12782.14, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-19", "l_commitdate": "1993-10-18", "l_receiptdate": "1993-12-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " requests. bravely i" }
+{ "l_orderkey": 5763, "l_partkey": 13, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22825.25, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-04", "l_commitdate": "1998-08-16", "l_receiptdate": "1998-10-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "inal theodolites. even re" }
+{ "l_orderkey": 5831, "l_partkey": 13, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 46.0, "l_extendedprice": 41998.46, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-24", "l_commitdate": "1997-01-18", "l_receiptdate": "1997-03-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ly final pa" }
+{ "l_orderkey": 5953, "l_partkey": 13, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 31042.34, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-04", "l_commitdate": "1992-06-12", "l_receiptdate": "1992-06-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "hockey players use furiously against th" }
+{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
+{ "l_orderkey": 130, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " slyly ironic decoys abou" }
 { "l_orderkey": 322, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 31920.35, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-05-03", "l_receiptdate": "1992-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egular accounts cajole carefully. even d" }
+{ "l_orderkey": 359, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "unusual warthogs. ironically sp" }
+{ "l_orderkey": 807, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10032.11, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts above the slyly final ex" }
+{ "l_orderkey": 928, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 34656.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "xpress grouc" }
+{ "l_orderkey": 1123, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9120.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-11-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckages are above the depths. slyly ir" }
+{ "l_orderkey": 1156, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26448.29, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts sleep sly" }
+{ "l_orderkey": 1345, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-12-11", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly express requests. ironic accounts c" }
+{ "l_orderkey": 1763, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20064.22, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-15", "l_receiptdate": "1997-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ld. fluffily final ideas boos" }
 { "l_orderkey": 1797, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19152.21, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-05", "l_commitdate": "1996-08-05", "l_receiptdate": "1996-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ns. regular, regular deposit" }
 { "l_orderkey": 2274, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-12-03", "l_receiptdate": "1993-09-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "usly final re" }
 { "l_orderkey": 2311, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 32.0, "l_extendedprice": 29184.32, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-19", "l_commitdate": "1995-06-26", "l_receiptdate": "1995-07-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "sts along the slyly" }
-{ "l_orderkey": 5762, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10944.12, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages are abo" }
-{ "l_orderkey": 5920, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25536.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-13", "l_receiptdate": "1994-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le slyly slyly even deposits. f" }
-{ "l_orderkey": 807, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 11.0, "l_extendedprice": 10032.11, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-14", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "unts above the slyly final ex" }
-{ "l_orderkey": 928, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 38.0, "l_extendedprice": 34656.38, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-08", "l_commitdate": "1995-04-15", "l_receiptdate": "1995-06-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "xpress grouc" }
-{ "l_orderkey": 1156, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26448.29, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ts sleep sly" }
-{ "l_orderkey": 1763, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20064.22, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-17", "l_commitdate": "1997-01-15", "l_receiptdate": "1997-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ld. fluffily final ideas boos" }
+{ "l_orderkey": 2435, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21888.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. carefully regular d" }
+{ "l_orderkey": 2497, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31008.34, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic accounts. p" }
 { "l_orderkey": 3204, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9120.1, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-27", "l_commitdate": "1993-03-08", "l_receiptdate": "1993-01-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "counts. bold " }
 { "l_orderkey": 3239, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28272.31, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-14", "l_commitdate": "1998-03-24", "l_receiptdate": "1998-04-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "foxes. pendin" }
 { "l_orderkey": 3715, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-03", "l_commitdate": "1996-04-30", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ut the carefully expr" }
 { "l_orderkey": 4038, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-17", "l_commitdate": "1996-03-19", "l_receiptdate": "1996-04-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": " packages " }
 { "l_orderkey": 4771, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4560.05, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-07", "l_commitdate": "1993-01-19", "l_receiptdate": "1993-01-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar, quiet accounts nag furiously express id" }
 { "l_orderkey": 5636, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 33.0, "l_extendedprice": 30096.33, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-09", "l_commitdate": "1995-04-05", "l_receiptdate": "1995-03-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ding to the " }
-{ "l_orderkey": 359, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-27", "l_commitdate": "1995-03-18", "l_receiptdate": "1995-01-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "unusual warthogs. ironically sp" }
-{ "l_orderkey": 1123, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9120.1, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-12", "l_commitdate": "1996-10-04", "l_receiptdate": "1996-11-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ckages are above the depths. slyly ir" }
-{ "l_orderkey": 1345, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33744.37, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-27", "l_commitdate": "1992-12-11", "l_receiptdate": "1992-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "e slyly express requests. ironic accounts c" }
-{ "l_orderkey": 2435, "l_partkey": 12, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21888.24, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-14", "l_commitdate": "1993-05-20", "l_receiptdate": "1993-03-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. carefully regular d" }
-{ "l_orderkey": 2497, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 31008.34, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-09-02", "l_commitdate": "1992-10-19", "l_receiptdate": "1992-09-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ronic accounts. p" }
-{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6.0, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
-{ "l_orderkey": 130, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16416.18, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-04", "l_commitdate": "1992-06-14", "l_receiptdate": "1992-07-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " slyly ironic decoys abou" }
 { "l_orderkey": 5729, "l_partkey": 12, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45600.5, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-09", "l_commitdate": "1994-12-31", "l_receiptdate": "1994-12-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ly special sentiments. car" }
-{ "l_orderkey": 773, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28241.31, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e slyly unusual deposit" }
-{ "l_orderkey": 2054, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 3644.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-12", "l_commitdate": "1992-08-31", "l_receiptdate": "1992-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lyly careful requests wake fl" }
-{ "l_orderkey": 2979, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42817.47, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-25", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously unusual dependencies wake across" }
-{ "l_orderkey": 4389, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20042.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lly silent de" }
-{ "l_orderkey": 4551, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5466.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily silent fo" }
-{ "l_orderkey": 4866, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8199.09, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-18", "l_receiptdate": "1997-09-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven dependencies x-ray. quic" }
+{ "l_orderkey": 5762, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10944.12, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-18", "l_commitdate": "1997-04-27", "l_receiptdate": "1997-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ages are abo" }
+{ "l_orderkey": 5920, "l_partkey": 12, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25536.28, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-17", "l_commitdate": "1995-02-13", "l_receiptdate": "1994-12-31", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "le slyly slyly even deposits. f" }
 { "l_orderkey": 103, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33707.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-17", "l_commitdate": "1996-07-27", "l_receiptdate": "1996-09-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ies. quickly ironic requests use blithely" }
+{ "l_orderkey": 198, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31885.35, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests nod quickly furiously sly pinto be" }
+{ "l_orderkey": 773, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28241.31, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-30", "l_commitdate": "1993-11-02", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "e slyly unusual deposit" }
+{ "l_orderkey": 928, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "longside of" }
 { "l_orderkey": 998, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 5466.06, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-20", "l_commitdate": "1994-12-27", "l_receiptdate": "1995-04-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "refully accounts. carefully express ac" }
+{ "l_orderkey": 1509, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously regula" }
+{ "l_orderkey": 2054, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 4.0, "l_extendedprice": 3644.04, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-12", "l_commitdate": "1992-08-31", "l_receiptdate": "1992-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "lyly careful requests wake fl" }
+{ "l_orderkey": 2147, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " the fluffily" }
 { "l_orderkey": 2466, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26419.29, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-04-20", "l_receiptdate": "1994-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es boost fluffily ab" }
 { "l_orderkey": 2469, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 48.0, "l_extendedprice": 43728.48, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-11", "l_commitdate": "1997-01-03", "l_receiptdate": "1997-01-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "riously even theodolites u" }
+{ "l_orderkey": 2979, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42817.47, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-25", "l_commitdate": "1996-05-13", "l_receiptdate": "1996-04-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "iously unusual dependencies wake across" }
+{ "l_orderkey": 3237, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-31", "l_receiptdate": "1992-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es. permanently express platelets besid" }
+{ "l_orderkey": 3460, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36440.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "odolites are slyly bold deposits" }
 { "l_orderkey": 3584, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 3644.04, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-16", "l_commitdate": "1997-10-31", "l_receiptdate": "1997-08-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "nal packag" }
 { "l_orderkey": 3872, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37351.41, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-23", "l_commitdate": "1996-11-12", "l_receiptdate": "1996-12-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly regular epitaphs boost" }
 { "l_orderkey": 4192, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32796.36, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-25", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-05-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "eodolites sleep" }
-{ "l_orderkey": 5573, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s haggle qu" }
-{ "l_orderkey": 5698, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27330.3, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its. quickly regular foxes aro" }
-{ "l_orderkey": 928, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-29", "l_commitdate": "1995-04-16", "l_receiptdate": "1995-04-30", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "longside of" }
-{ "l_orderkey": 2147, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-27", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-10-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " the fluffily" }
 { "l_orderkey": 4263, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 38.0, "l_extendedprice": 34618.38, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-05-08", "l_receiptdate": "1998-07-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "rding to the dep" }
+{ "l_orderkey": 4389, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 22.0, "l_extendedprice": 20042.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-07-12", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "lly silent de" }
+{ "l_orderkey": 4551, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5466.06, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-18", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-06-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "fily silent fo" }
 { "l_orderkey": 4800, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19131.21, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-14", "l_commitdate": "1992-03-15", "l_receiptdate": "1992-02-26", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ithely according to " }
+{ "l_orderkey": 4866, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 8199.09, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-30", "l_commitdate": "1997-09-18", "l_receiptdate": "1997-09-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ven dependencies x-ray. quic" }
 { "l_orderkey": 4934, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 2.0, "l_extendedprice": 1822.02, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-05", "l_commitdate": "1997-03-26", "l_receiptdate": "1997-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ongside of the brave, regula" }
 { "l_orderkey": 4935, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21864.24, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-29", "l_commitdate": "1993-08-17", "l_receiptdate": "1993-06-22", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly quickly s" }
-{ "l_orderkey": 5633, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 35529.39, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ding ideas cajole furiously after" }
-{ "l_orderkey": 198, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31885.35, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-27", "l_commitdate": "1998-03-23", "l_receiptdate": "1998-03-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ests nod quickly furiously sly pinto be" }
-{ "l_orderkey": 1509, "l_partkey": 11, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-15", "l_commitdate": "1993-10-04", "l_receiptdate": "1993-11-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "uriously regula" }
-{ "l_orderkey": 3237, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 11.0, "l_extendedprice": 10021.11, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-03", "l_commitdate": "1992-07-31", "l_receiptdate": "1992-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "es. permanently express platelets besid" }
-{ "l_orderkey": 3460, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 36440.4, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-28", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "odolites are slyly bold deposits" }
 { "l_orderkey": 5187, "l_partkey": 11, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 44639.49, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-20", "l_commitdate": "1997-10-12", "l_receiptdate": "1997-10-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "l, regular platelets instead of the foxes w" }
+{ "l_orderkey": 5573, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46.0, "l_extendedprice": 41906.46, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-04", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-11-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s haggle qu" }
+{ "l_orderkey": 5633, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 39.0, "l_extendedprice": 35529.39, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-12", "l_commitdate": "1998-07-03", "l_receiptdate": "1998-07-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ding ideas cajole furiously after" }
+{ "l_orderkey": 5698, "l_partkey": 11, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27330.3, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-16", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "its. quickly regular foxes aro" }
 { "l_orderkey": 5858, "l_partkey": 11, "l_suppkey": 5, "l_linenumber": 7, "l_quantity": 50.0, "l_extendedprice": 45550.5, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-20", "l_commitdate": "1992-10-07", "l_receiptdate": "1992-07-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "r the ironic ex" }
-{ "l_orderkey": 994, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10010.11, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-05-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accounts sleep " }
-{ "l_orderkey": 1382, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-17", "l_commitdate": "1993-09-28", "l_receiptdate": "1993-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake pending pinto beans. s" }
-{ "l_orderkey": 1922, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11830.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-11-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quests. furiously" }
-{ "l_orderkey": 2976, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31850.35, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "boost slyly about the regular, regular re" }
-{ "l_orderkey": 3363, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38220.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " blithely final ideas nag after" }
-{ "l_orderkey": 3558, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l deposits " }
-{ "l_orderkey": 5507, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20930.23, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-07-04", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ously slow packages poach whithout the" }
-{ "l_orderkey": 998, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20020.22, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-02-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lites. qui" }
-{ "l_orderkey": 1668, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep slyly across the furi" }
-{ "l_orderkey": 1796, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly ironic accounts." }
-{ "l_orderkey": 2980, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43680.48, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "totes. regular pinto " }
-{ "l_orderkey": 4161, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40950.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "about the ironic packages cajole blithe" }
-{ "l_orderkey": 4807, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37310.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-31", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " fluffily re" }
-{ "l_orderkey": 4999, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40040.44, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ependencies. slowly regu" }
+{ "l_orderkey": 196, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13650.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s accounts. furio" }
 { "l_orderkey": 230, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 44.0, "l_extendedprice": 40040.44, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-09", "l_commitdate": "1994-01-18", "l_receiptdate": "1994-03-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "deposits integrate slyly sile" }
 { "l_orderkey": 449, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2730.03, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-28", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " bold deposits. express theodolites haggle" }
 { "l_orderkey": 519, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 11830.13, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-06", "l_commitdate": "1997-12-02", "l_receiptdate": "1998-03-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "c accounts wake along the ironic so" }
+{ "l_orderkey": 994, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 10010.11, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-06-10", "l_receiptdate": "1994-05-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ular accounts sleep " }
+{ "l_orderkey": 998, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 20020.22, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-03", "l_commitdate": "1995-02-17", "l_receiptdate": "1994-12-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "lites. qui" }
+{ "l_orderkey": 1382, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-17", "l_commitdate": "1993-09-28", "l_receiptdate": "1993-11-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ake pending pinto beans. s" }
+{ "l_orderkey": 1668, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 38.0, "l_extendedprice": 34580.38, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-26", "l_commitdate": "1997-09-17", "l_receiptdate": "1997-09-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ep slyly across the furi" }
 { "l_orderkey": 1730, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36400.4, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-02", "l_commitdate": "1998-10-06", "l_receiptdate": "1998-10-03", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ven dinos slee" }
-{ "l_orderkey": 2946, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22750.25, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-06", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic deposits. furiously" }
-{ "l_orderkey": 2947, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33670.37, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e accounts: expres" }
-{ "l_orderkey": 4102, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15470.17, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly silent theodolites sleep unusual exc" }
-{ "l_orderkey": 4935, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12740.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slowly. blith" }
-{ "l_orderkey": 196, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13650.15, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-05", "l_commitdate": "1993-05-08", "l_receiptdate": "1993-07-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s accounts. furio" }
+{ "l_orderkey": 1796, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-01", "l_commitdate": "1993-01-01", "l_receiptdate": "1992-12-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly ironic accounts." }
+{ "l_orderkey": 1922, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11830.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-09-21", "l_receiptdate": "1996-11-15", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "quests. furiously" }
 { "l_orderkey": 1989, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 47.0, "l_extendedprice": 42770.47, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-21", "l_commitdate": "1994-05-27", "l_receiptdate": "1994-06-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "final deposits s" }
 { "l_orderkey": 2881, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 910.01, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-13", "l_commitdate": "1992-07-21", "l_receiptdate": "1992-05-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "final theodolites. quickly" }
+{ "l_orderkey": 2946, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22750.25, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-06", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-05-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ic deposits. furiously" }
+{ "l_orderkey": 2947, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33670.37, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-09", "l_commitdate": "1995-07-05", "l_receiptdate": "1995-08-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "e accounts: expres" }
+{ "l_orderkey": 2976, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31850.35, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-19", "l_commitdate": "1994-02-14", "l_receiptdate": "1994-01-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "boost slyly about the regular, regular re" }
+{ "l_orderkey": 2980, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43680.48, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-12-09", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "totes. regular pinto " }
 { "l_orderkey": 3138, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 12.0, "l_extendedprice": 10920.12, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-04-11", "l_receiptdate": "1994-03-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": ". bold pinto beans haggl" }
 { "l_orderkey": 3141, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33670.37, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-24", "l_commitdate": "1995-12-16", "l_receiptdate": "1996-01-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "press pinto beans. bold accounts boost b" }
+{ "l_orderkey": 3363, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 38220.42, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-09", "l_commitdate": "1995-11-25", "l_receiptdate": "1995-11-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " blithely final ideas nag after" }
+{ "l_orderkey": 3558, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25480.28, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-02", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "l deposits " }
 { "l_orderkey": 3941, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 2.0, "l_extendedprice": 1820.02, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1996-10-01", "l_receiptdate": "1996-12-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "es wake after the" }
+{ "l_orderkey": 4102, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15470.17, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-03", "l_commitdate": "1996-05-06", "l_receiptdate": "1996-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "ly silent theodolites sleep unusual exc" }
+{ "l_orderkey": 4161, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40950.45, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-22", "l_commitdate": "1993-10-17", "l_receiptdate": "1993-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "about the ironic packages cajole blithe" }
+{ "l_orderkey": 4807, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37310.41, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-02", "l_commitdate": "1997-03-31", "l_receiptdate": "1997-05-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " fluffily re" }
 { "l_orderkey": 4839, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22750.25, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-20", "l_commitdate": "1994-07-08", "l_receiptdate": "1994-05-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "regular packages ab" }
+{ "l_orderkey": 4935, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12740.14, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-05-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "slowly. blith" }
+{ "l_orderkey": 4999, "l_partkey": 10, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 44.0, "l_extendedprice": 40040.44, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-01", "l_commitdate": "1993-08-04", "l_receiptdate": "1993-08-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ependencies. slowly regu" }
 { "l_orderkey": 5378, "l_partkey": 10, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16380.18, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-25", "l_commitdate": "1992-12-21", "l_receiptdate": "1992-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic accounts was bold, " }
+{ "l_orderkey": 5507, "l_partkey": 10, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20930.23, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-07-04", "l_receiptdate": "1998-09-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ously slow packages poach whithout the" }
 { "l_orderkey": 5632, "l_partkey": 10, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 48.0, "l_extendedprice": 43680.48, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-03-24", "l_receiptdate": "1996-06-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "unts. decoys u" }
-{ "l_orderkey": 903, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 29997.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely ironic packages wake blithely" }
-{ "l_orderkey": 1152, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20907.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "equests alongside of the unusual " }
-{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nts. fluffily special instructions integr" }
-{ "l_orderkey": 2055, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13635.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-21", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular foxes. b" }
-{ "l_orderkey": 2976, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29088.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nding, ironic deposits sleep f" }
-{ "l_orderkey": 4196, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28179.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the blithely ironic inst" }
-{ "l_orderkey": 4483, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45450.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. furiously ironi" }
-{ "l_orderkey": 5221, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30906.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-07-17", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eans. furio" }
-{ "l_orderkey": 5414, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17271.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ffily silent theodolites na" }
 { "l_orderkey": 293, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12726.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-12-23", "l_receiptdate": "1992-11-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es. packages above the" }
 { "l_orderkey": 419, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 15.0, "l_extendedprice": 13635.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-09", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "of the careful, thin theodolites. quickly s" }
 { "l_orderkey": 903, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 35.0, "l_extendedprice": 31815.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-18", "l_commitdate": "1995-08-21", "l_receiptdate": "1995-10-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "rets wake fin" }
-{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4545.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely regular accounts are slyly. pending, bo" }
-{ "l_orderkey": 2979, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "st blithely; blithely regular gifts dazz" }
-{ "l_orderkey": 4416, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40905.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the final pinto beans. special frets " }
-{ "l_orderkey": 4771, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-22", "l_receiptdate": "1992-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " carefully re" }
+{ "l_orderkey": 903, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 29997.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-24", "l_commitdate": "1995-09-01", "l_receiptdate": "1995-10-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ely ironic packages wake blithely" }
+{ "l_orderkey": 1152, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20907.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-14", "l_commitdate": "1994-10-22", "l_receiptdate": "1994-10-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "equests alongside of the unusual " }
 { "l_orderkey": 1568, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 46.0, "l_extendedprice": 41814.0, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-06", "l_commitdate": "1997-04-08", "l_receiptdate": "1997-04-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "g the blithely even acco" }
+{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4545.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-13", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ely regular accounts are slyly. pending, bo" }
+{ "l_orderkey": 1792, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-21", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-02-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "nts. fluffily special instructions integr" }
 { "l_orderkey": 1955, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 14544.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-30", "l_commitdate": "1992-06-23", "l_receiptdate": "1992-05-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "odolites eat s" }
+{ "l_orderkey": 2055, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13635.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-30", "l_commitdate": "1993-11-21", "l_receiptdate": "1993-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "gular foxes. b" }
 { "l_orderkey": 2693, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23634.0, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-14", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "cajole alo" }
-{ "l_orderkey": 3970, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 41814.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-05-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly ironic" }
-{ "l_orderkey": 4096, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes mold flu" }
-{ "l_orderkey": 5536, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27270.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-05-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "arefully regular theodolites according" }
+{ "l_orderkey": 2976, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 29088.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-26", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-02-10", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "nding, ironic deposits sleep f" }
+{ "l_orderkey": 2979, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 7272.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-18", "l_commitdate": "1996-05-21", "l_receiptdate": "1996-07-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "st blithely; blithely regular gifts dazz" }
 { "l_orderkey": 3075, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35451.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-10", "l_commitdate": "1994-06-21", "l_receiptdate": "1994-06-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ing deposits nag " }
 { "l_orderkey": 3233, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22725.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-24", "l_commitdate": "1995-01-07", "l_receiptdate": "1994-12-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "oss the pl" }
+{ "l_orderkey": 3970, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 41814.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-05-14", "l_receiptdate": "1992-05-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "yly ironic" }
+{ "l_orderkey": 4096, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-08-24", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "tes mold flu" }
+{ "l_orderkey": 4196, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 28179.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-12", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ut the blithely ironic inst" }
 { "l_orderkey": 4199, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 18.0, "l_extendedprice": 16362.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-01", "l_commitdate": "1992-03-30", "l_receiptdate": "1992-06-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "pending, regular accounts. carefully" }
+{ "l_orderkey": 4416, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40905.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-16", "l_commitdate": "1992-09-09", "l_receiptdate": "1992-10-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "the final pinto beans. special frets " }
+{ "l_orderkey": 4483, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45450.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-10", "l_commitdate": "1992-04-18", "l_receiptdate": "1992-06-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ackages. furiously ironi" }
 { "l_orderkey": 4545, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 36.0, "l_extendedprice": 32724.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-28", "l_commitdate": "1993-03-30", "l_receiptdate": "1993-02-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "sublate slyly. furiously ironic accounts b" }
+{ "l_orderkey": 4771, "l_partkey": 9, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19089.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-20", "l_commitdate": "1993-01-22", "l_receiptdate": "1992-12-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " carefully re" }
 { "l_orderkey": 4805, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 38178.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-17", "l_commitdate": "1992-07-03", "l_receiptdate": "1992-09-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "the regular, fina" }
+{ "l_orderkey": 5221, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30906.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-11", "l_commitdate": "1995-07-17", "l_receiptdate": "1995-10-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "eans. furio" }
+{ "l_orderkey": 5414, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17271.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-06", "l_commitdate": "1993-05-12", "l_receiptdate": "1993-05-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ffily silent theodolites na" }
 { "l_orderkey": 5511, "l_partkey": 9, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 20907.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-21", "l_receiptdate": "1995-03-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ing dugouts " }
+{ "l_orderkey": 5536, "l_partkey": 9, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 27270.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-15", "l_commitdate": "1998-05-23", "l_receiptdate": "1998-05-03", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "arefully regular theodolites according" }
 { "l_orderkey": 5859, "l_partkey": 9, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15453.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-15", "l_commitdate": "1997-06-30", "l_receiptdate": "1997-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly ironic requests. quickly unusual pin" }
 { "l_orderkey": 68, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully special instructions cajole. furious" }
 { "l_orderkey": 225, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28148.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "special platelets. quickly r" }
 { "l_orderkey": 230, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 908.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely unusual dolphins. bold, ex" }
-{ "l_orderkey": 1283, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 27240.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-22", "l_receiptdate": "1996-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t the fluffily" }
-{ "l_orderkey": 1472, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "riously silent deposits to the pending d" }
-{ "l_orderkey": 2277, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1816.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "endencies sleep idly pending p" }
-{ "l_orderkey": 4613, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-22", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gainst the furiously ironic" }
-{ "l_orderkey": 5478, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35412.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-09-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. furiously " }
-{ "l_orderkey": 5600, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17252.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dencies. carefully p" }
-{ "l_orderkey": 5894, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20884.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-09-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " furiously even deposits haggle alw" }
-{ "l_orderkey": 1760, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly bold dolphins haggle carefully. sl" }
-{ "l_orderkey": 2048, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4540.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "affix carefully against " }
-{ "l_orderkey": 3777, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9080.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le. ironic depths a" }
-{ "l_orderkey": 4322, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10896.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e blithely against the slyly unusu" }
-{ "l_orderkey": 5057, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-10-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " asymptotes wake slyl" }
 { "l_orderkey": 1060, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-12", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts; even deposits are carefull" }
 { "l_orderkey": 1222, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-13", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ", even accounts are ironic" }
-{ "l_orderkey": 1828, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " accounts run slyly " }
-{ "l_orderkey": 4291, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uctions. furiously regular ins" }
-{ "l_orderkey": 4420, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6356.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular instructions sleep around" }
-{ "l_orderkey": 5635, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36320.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending foxes. regular packages" }
+{ "l_orderkey": 1283, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 27240.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-22", "l_receiptdate": "1996-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t the fluffily" }
+{ "l_orderkey": 1472, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "riously silent deposits to the pending d" }
 { "l_orderkey": 1540, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-15", "l_commitdate": "1992-10-24", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ironic deposits amo" }
+{ "l_orderkey": 1760, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly bold dolphins haggle carefully. sl" }
 { "l_orderkey": 1762, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44492.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-20", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages sleep fluffily pen" }
+{ "l_orderkey": 1828, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " accounts run slyly " }
+{ "l_orderkey": 2048, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4540.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "affix carefully against " }
+{ "l_orderkey": 2277, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1816.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "endencies sleep idly pending p" }
+{ "l_orderkey": 3777, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9080.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le. ironic depths a" }
+{ "l_orderkey": 4291, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uctions. furiously regular ins" }
+{ "l_orderkey": 4322, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10896.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e blithely against the slyly unusu" }
+{ "l_orderkey": 4420, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6356.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular instructions sleep around" }
+{ "l_orderkey": 4613, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-22", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gainst the furiously ironic" }
 { "l_orderkey": 4614, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-26", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic foxes affix furi" }
+{ "l_orderkey": 5057, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-20", "l_commitdate": "1997-10-02", "l_receiptdate": "1997-10-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " asymptotes wake slyl" }
+{ "l_orderkey": 5478, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35412.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-19", "l_commitdate": "1996-06-25", "l_receiptdate": "1996-09-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "s. furiously " }
+{ "l_orderkey": 5600, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 17252.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-10", "l_commitdate": "1997-03-24", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "dencies. carefully p" }
+{ "l_orderkey": 5635, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 36320.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-25", "l_commitdate": "1992-11-05", "l_receiptdate": "1992-10-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pending foxes. regular packages" }
+{ "l_orderkey": 5894, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20884.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-05", "l_commitdate": "1994-10-27", "l_receiptdate": "1994-09-13", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " furiously even deposits haggle alw" }
+{ "l_orderkey": 579, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37187.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold, express requests sublate slyly. blith" }
 { "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-18", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully. pending in" }
 { "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 12698.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-31", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r, final packages are slyly iro" }
+{ "l_orderkey": 834, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9977.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inst the regular packa" }
 { "l_orderkey": 1281, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-02-02", "l_receiptdate": "1995-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ounts detect" }
 { "l_orderkey": 2080, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4535.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-26", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "refully unusual theo" }
-{ "l_orderkey": 2752, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26303.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gly blithely re" }
-{ "l_orderkey": 3140, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19047.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furiously sly excuses according to the" }
-{ "l_orderkey": 3204, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35373.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-19", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sits sleep theodolites. slyly bo" }
-{ "l_orderkey": 5606, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22675.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "breach about the furiously bold " }
-{ "l_orderkey": 579, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37187.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold, express requests sublate slyly. blith" }
-{ "l_orderkey": 2598, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-17", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "express packages nag sly" }
-{ "l_orderkey": 4166, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15419.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages. re" }
-{ "l_orderkey": 5794, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13605.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-27", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular ideas. final foxes haggle " }
-{ "l_orderkey": 834, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9977.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inst the regular packa" }
 { "l_orderkey": 2150, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "press platelets haggle until the slyly fi" }
 { "l_orderkey": 2182, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ments are fu" }
 { "l_orderkey": 2208, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-05-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e fluffily regular theodolites caj" }
-{ "l_orderkey": 5318, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ickly final deposi" }
-{ "l_orderkey": 5351, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss the ironic, regular asymptotes cajole " }
-{ "l_orderkey": 5670, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21768.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "press, express requests haggle" }
+{ "l_orderkey": 2598, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-17", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "express packages nag sly" }
 { "l_orderkey": 2658, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-02", "l_commitdate": "1995-11-08", "l_receiptdate": "1995-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e special requests. quickly ex" }
 { "l_orderkey": 2659, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8163.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-07", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly final packages sleep ac" }
+{ "l_orderkey": 2752, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26303.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gly blithely re" }
+{ "l_orderkey": 3140, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19047.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furiously sly excuses according to the" }
+{ "l_orderkey": 3204, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35373.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-19", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sits sleep theodolites. slyly bo" }
+{ "l_orderkey": 4166, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15419.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages. re" }
 { "l_orderkey": 4614, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 17233.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-06-21", "l_receiptdate": "1996-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ix. carefully regular " }
+{ "l_orderkey": 5318, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ickly final deposi" }
+{ "l_orderkey": 5351, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss the ironic, regular asymptotes cajole " }
+{ "l_orderkey": 5606, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22675.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "breach about the furiously bold " }
+{ "l_orderkey": 5670, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21768.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "press, express requests haggle" }
+{ "l_orderkey": 5794, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13605.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-27", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular ideas. final foxes haggle " }
+{ "l_orderkey": 260, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26274.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fluffily even asymptotes. express wa" }
 { "l_orderkey": 290, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 31710.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-04-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ove the final foxes detect slyly fluffily" }
+{ "l_orderkey": 768, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27180.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously fluffy pinto beans haggle along" }
 { "l_orderkey": 801, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are fluffily stealthily expres" }
 { "l_orderkey": 1124, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "t the slyly " }
+{ "l_orderkey": 1220, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32616.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unusual, silent pinto beans aga" }
 { "l_orderkey": 1284, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " regular asymptotes. " }
+{ "l_orderkey": 1542, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending instr" }
 { "l_orderkey": 1638, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-10-28", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "otes haggle before the slyly bold instructi" }
 { "l_orderkey": 1827, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-17", "l_commitdate": "1996-08-29", "l_receiptdate": "1996-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely. express, bo" }
 { "l_orderkey": 2049, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35334.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the even pinto beans " }
-{ "l_orderkey": 4483, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 28992.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests haggle. slyl" }
-{ "l_orderkey": 5957, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es across the regular requests maint" }
+{ "l_orderkey": 2054, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n pinto beans. ironic courts are iro" }
+{ "l_orderkey": 2179, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " cajole carefully. " }
+{ "l_orderkey": 2276, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-30", "l_receiptdate": "1996-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. deposits " }
+{ "l_orderkey": 2370, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19026.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ecial dependencies must have to " }
+{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40770.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
+{ "l_orderkey": 3043, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13590.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "usly furiously" }
+{ "l_orderkey": 3137, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. silent excuses boost about" }
 { "l_orderkey": 3329, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final depo" }
 { "l_orderkey": 3426, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pecial theodolites haggle fluf" }
 { "l_orderkey": 4005, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld requests. slyly final instructi" }
+{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
+{ "l_orderkey": 4195, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. carefully express" }
+{ "l_orderkey": 4387, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the blithely regular fox" }
+{ "l_orderkey": 4483, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 28992.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests haggle. slyl" }
+{ "l_orderkey": 4485, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 42582.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffily pending acc" }
+{ "l_orderkey": 4612, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18120.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans sleep blithely iro" }
 { "l_orderkey": 4676, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 29898.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-29", "l_commitdate": "1995-10-01", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly express " }
 { "l_orderkey": 4870, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its wake quickly. slyly quick" }
 { "l_orderkey": 5124, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-05", "l_commitdate": "1997-06-29", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "wake across the" }
 { "l_orderkey": 5125, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ily even deposits w" }
-{ "l_orderkey": 260, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26274.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fluffily even asymptotes. express wa" }
-{ "l_orderkey": 768, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27180.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously fluffy pinto beans haggle along" }
-{ "l_orderkey": 1542, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending instr" }
-{ "l_orderkey": 2179, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " cajole carefully. " }
-{ "l_orderkey": 2276, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-30", "l_receiptdate": "1996-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. deposits " }
-{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40770.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
-{ "l_orderkey": 4485, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 42582.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffily pending acc" }
-{ "l_orderkey": 4612, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18120.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans sleep blithely iro" }
-{ "l_orderkey": 5760, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s. bravely ironic accounts among" }
-{ "l_orderkey": 1220, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32616.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unusual, silent pinto beans aga" }
-{ "l_orderkey": 2054, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n pinto beans. ironic courts are iro" }
-{ "l_orderkey": 2370, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19026.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ecial dependencies must have to " }
-{ "l_orderkey": 3043, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13590.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "usly furiously" }
-{ "l_orderkey": 3137, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. silent excuses boost about" }
-{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
-{ "l_orderkey": 4195, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. carefully express" }
-{ "l_orderkey": 4387, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the blithely regular fox" }
 { "l_orderkey": 5223, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17214.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-28", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntly. furiously even excuses a" }
+{ "l_orderkey": 5760, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s. bravely ironic accounts among" }
+{ "l_orderkey": 5957, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es across the regular requests maint" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 228, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2715.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckages. sly" }
 { "l_orderkey": 320, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27150.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic, final accounts wake quick de" }
+{ "l_orderkey": 354, "l_partkey": 5, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t thinly above the ironic, " }
+{ "l_orderkey": 548, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5430.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits wake furiously regular" }
+{ "l_orderkey": 645, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 38915.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously accounts. slyly" }
+{ "l_orderkey": 675, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits along the express foxes " }
 { "l_orderkey": 966, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial ins" }
 { "l_orderkey": 993, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "sits. pending pinto beans haggle? ca" }
 { "l_orderkey": 1058, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the final requests believe carefully " }
-{ "l_orderkey": 1829, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9955.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding orbits" }
-{ "l_orderkey": 2819, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25340.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages sublate carefully closely regular " }
-{ "l_orderkey": 3076, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 28055.0, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular depos" }
-{ "l_orderkey": 354, "l_partkey": 5, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t thinly above the ironic, " }
-{ "l_orderkey": 645, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 38915.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously accounts. slyly" }
+{ "l_orderkey": 1155, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44345.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-12-30", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts are alongside of t" }
+{ "l_orderkey": 1574, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 38010.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans according t" }
 { "l_orderkey": 1633, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-13", "l_commitdate": "1995-11-13", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ges wake fluffil" }
 { "l_orderkey": 1637, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " haggle carefully silent accou" }
+{ "l_orderkey": 1732, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45250.0, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-23", "l_receiptdate": "1993-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily final asymptotes according " }
+{ "l_orderkey": 1829, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9955.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding orbits" }
+{ "l_orderkey": 2178, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36200.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes are slowly regularly specia" }
+{ "l_orderkey": 2241, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " final deposits use fluffily. even f" }
+{ "l_orderkey": 2375, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4525.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "final packages cajole according to the furi" }
 { "l_orderkey": 2725, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns sleep furiously c" }
+{ "l_orderkey": 2819, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25340.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages sublate carefully closely regular " }
+{ "l_orderkey": 3076, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 28055.0, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular depos" }
 { "l_orderkey": 3328, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 20815.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-12", "l_commitdate": "1993-02-07", "l_receiptdate": "1993-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. careful" }
+{ "l_orderkey": 3555, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17195.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep special theodolit" }
+{ "l_orderkey": 3649, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "special re" }
 { "l_orderkey": 3680, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-01-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "iously ironic platelets in" }
 { "l_orderkey": 3877, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-30", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "integrate against the expres" }
-{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
-{ "l_orderkey": 228, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2715.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckages. sly" }
-{ "l_orderkey": 548, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5430.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits wake furiously regular" }
-{ "l_orderkey": 675, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits along the express foxes " }
-{ "l_orderkey": 1155, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44345.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-12-30", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts are alongside of t" }
-{ "l_orderkey": 1732, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45250.0, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-23", "l_receiptdate": "1993-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily final asymptotes according " }
-{ "l_orderkey": 2375, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4525.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "final packages cajole according to the furi" }
 { "l_orderkey": 3970, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-05-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ix slyly. quickly silen" }
 { "l_orderkey": 4229, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30770.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-04-13", "l_receiptdate": "1998-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thely final accounts use even packa" }
 { "l_orderkey": 5283, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-16", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al deposits? blithely even pinto beans" }
-{ "l_orderkey": 1574, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 38010.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans according t" }
-{ "l_orderkey": 2178, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36200.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes are slowly regularly specia" }
-{ "l_orderkey": 2241, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " final deposits use fluffily. even f" }
-{ "l_orderkey": 3555, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17195.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep special theodolit" }
-{ "l_orderkey": 3649, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "special re" }
 { "l_orderkey": 5665, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "- special pinto beans sleep quickly blithel" }
 { "l_orderkey": 5959, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3620.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-07-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests ar" }
-{ "l_orderkey": 704, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the quickly final forges. furiously p" }
-{ "l_orderkey": 2279, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing foxes above the even accounts use slyly" }
-{ "l_orderkey": 2593, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the furiously " }
-{ "l_orderkey": 2885, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-12-12", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions solve. slyly regular requests n" }
-{ "l_orderkey": 4292, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 42488.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y packages; even ideas boost" }
-{ "l_orderkey": 4736, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38872.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1995-12-21", "l_receiptdate": "1996-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. carefully " }
-{ "l_orderkey": 5349, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-10-08", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits affix carefully" }
-{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 904.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
-{ "l_orderkey": 641, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37064.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-27", "l_receiptdate": "1993-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " asymptotes are quickly. bol" }
-{ "l_orderkey": 2019, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28024.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l ideas across the slowl" }
-{ "l_orderkey": 2144, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-05-16", "l_receiptdate": "1994-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully ironic" }
-{ "l_orderkey": 2560, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24408.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " against the carefully" }
-{ "l_orderkey": 2976, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21696.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic pinto beans. slyly bol" }
-{ "l_orderkey": 4099, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slowly final warthogs sleep blithely. q" }
-{ "l_orderkey": 5411, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17176.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ial accounts according to the f" }
 { "l_orderkey": 164, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 20792.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ress packages haggle ideas. blithely spec" }
+{ "l_orderkey": 641, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37064.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-27", "l_receiptdate": "1993-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " asymptotes are quickly. bol" }
+{ "l_orderkey": 704, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the quickly final forges. furiously p" }
 { "l_orderkey": 739, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 45200.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-26", "l_commitdate": "1998-07-16", "l_receiptdate": "1998-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ndencies. blith" }
 { "l_orderkey": 1251, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33448.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-21", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". furiously" }
 { "l_orderkey": 1316, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6328.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". furiously even accounts a" }
+{ "l_orderkey": 2019, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28024.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l ideas across the slowl" }
+{ "l_orderkey": 2144, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-05-16", "l_receiptdate": "1994-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully ironic" }
+{ "l_orderkey": 2279, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing foxes above the even accounts use slyly" }
 { "l_orderkey": 2404, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16272.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-25", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-07-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "packages. even requests according to " }
+{ "l_orderkey": 2560, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24408.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " against the carefully" }
+{ "l_orderkey": 2593, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the furiously " }
 { "l_orderkey": 2790, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28928.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-25", "l_commitdate": "1994-10-26", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ully pending" }
-{ "l_orderkey": 3522, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1994-12-09", "l_receiptdate": "1995-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes snooze " }
-{ "l_orderkey": 4196, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular packages haggle furiously alongs" }
-{ "l_orderkey": 5668, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13560.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the express, pending requests. bo" }
 { "l_orderkey": 2882, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kly. even requests w" }
+{ "l_orderkey": 2885, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-12-12", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions solve. slyly regular requests n" }
+{ "l_orderkey": 2976, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21696.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic pinto beans. slyly bol" }
+{ "l_orderkey": 3522, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1994-12-09", "l_receiptdate": "1995-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes snooze " }
 { "l_orderkey": 4005, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23504.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1997-02-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " to the quic" }
+{ "l_orderkey": 4099, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slowly final warthogs sleep blithely. q" }
+{ "l_orderkey": 4196, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular packages haggle furiously alongs" }
+{ "l_orderkey": 4292, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 42488.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y packages; even ideas boost" }
+{ "l_orderkey": 4736, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38872.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1995-12-21", "l_receiptdate": "1996-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. carefully " }
+{ "l_orderkey": 5349, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-10-08", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits affix carefully" }
+{ "l_orderkey": 5411, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17176.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ial accounts according to the f" }
+{ "l_orderkey": 5668, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13560.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the express, pending requests. bo" }
+{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 904.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
 { "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 39, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 39732.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eodolites. careful" }
 { "l_orderkey": 129, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41538.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-01-24", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uietly bold theodolites. fluffil" }
 { "l_orderkey": 194, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular deposi" }
 { "l_orderkey": 519, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34314.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1997-12-15", "l_receiptdate": "1998-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular excuses detect quickly furiously " }
+{ "l_orderkey": 801, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18963.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cial, special packages." }
+{ "l_orderkey": 993, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lites. even theodolite" }
 { "l_orderkey": 999, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9030.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-23", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully pending" }
+{ "l_orderkey": 1186, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily spec" }
+{ "l_orderkey": 1508, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cording to the furiously ironic depe" }
 { "l_orderkey": 1542, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 10836.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "carefully " }
+{ "l_orderkey": 2372, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xcuses. slyly ironic theod" }
 { "l_orderkey": 2401, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 44247.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "lites cajole carefully " }
 { "l_orderkey": 2721, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1806.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-13", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " slyly final requests against " }
+{ "l_orderkey": 2946, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31605.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-15", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sublate along the fluffily iron" }
+{ "l_orderkey": 2951, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans wake ac" }
+{ "l_orderkey": 3015, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " the furiously pendi" }
+{ "l_orderkey": 3110, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 30702.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly pending requests ha" }
 { "l_orderkey": 3137, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5418.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-10-23", "l_receiptdate": "1995-10-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly express as" }
+{ "l_orderkey": 3331, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23478.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-17", "l_receiptdate": "1993-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "p asymptotes. carefully unusual in" }
+{ "l_orderkey": 3776, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35217.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "yly blithely pending packages" }
 { "l_orderkey": 3937, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26187.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-22", "l_receiptdate": "1998-03-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nt pinto beans above the pending instr" }
-{ "l_orderkey": 39, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 39732.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eodolites. careful" }
-{ "l_orderkey": 801, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18963.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cial, special packages." }
-{ "l_orderkey": 1508, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cording to the furiously ironic depe" }
+{ "l_orderkey": 4484, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 37926.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding, pending requests wake. fluffily " }
 { "l_orderkey": 4740, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19866.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-08-17", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final dependencies nag " }
 { "l_orderkey": 5252, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37023.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ording to the blithely express somas sho" }
 { "l_orderkey": 5892, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ithely unusual accounts will have to integ" }
-{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
-{ "l_orderkey": 993, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lites. even theodolite" }
-{ "l_orderkey": 1186, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily spec" }
-{ "l_orderkey": 2372, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xcuses. slyly ironic theod" }
-{ "l_orderkey": 2946, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31605.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-15", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sublate along the fluffily iron" }
-{ "l_orderkey": 3015, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " the furiously pendi" }
-{ "l_orderkey": 3110, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 30702.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly pending requests ha" }
-{ "l_orderkey": 3331, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23478.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-17", "l_receiptdate": "1993-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "p asymptotes. carefully unusual in" }
-{ "l_orderkey": 2951, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans wake ac" }
-{ "l_orderkey": 3776, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35217.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "yly blithely pending packages" }
-{ "l_orderkey": 4484, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 37926.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding, pending requests wake. fluffily " }
+{ "l_orderkey": 65, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18942.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bove the even packages. accounts nag carefu" }
+{ "l_orderkey": 130, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43296.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely alongside of the regu" }
 { "l_orderkey": 261, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30668.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "c packages. asymptotes da" }
+{ "l_orderkey": 290, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4510.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ans integrate. requests sleep. fur" }
+{ "l_orderkey": 418, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "regular, silent pinto" }
 { "l_orderkey": 740, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "odolites cajole ironic, pending instruc" }
-{ "l_orderkey": 1575, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10824.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold accounts. furi" }
-{ "l_orderkey": 3362, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2706.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-26", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its cajole blithely excuses. de" }
-{ "l_orderkey": 3650, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-07-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re about the pinto " }
-{ "l_orderkey": 3654, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts doze bravely ab" }
-{ "l_orderkey": 4032, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24354.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously according to" }
-{ "l_orderkey": 4387, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s hinder quietly across the pla" }
-{ "l_orderkey": 5090, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-03", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ular requests su" }
-{ "l_orderkey": 5862, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26158.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e fluffily. furiously" }
 { "l_orderkey": 896, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6314.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-02", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " requests " }
+{ "l_orderkey": 1250, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, i" }
+{ "l_orderkey": 1575, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10824.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold accounts. furi" }
 { "l_orderkey": 1696, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its maintain alongside of the f" }
 { "l_orderkey": 2370, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-04-09", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "final depen" }
 { "l_orderkey": 2437, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s deposits. pendi" }
-{ "l_orderkey": 5478, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42394.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " instructions; slyly even accounts hagg" }
-{ "l_orderkey": 5828, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25256.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special ideas haggle slyly ac" }
-{ "l_orderkey": 65, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18942.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bove the even packages. accounts nag carefu" }
-{ "l_orderkey": 130, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43296.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely alongside of the regu" }
-{ "l_orderkey": 290, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4510.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ans integrate. requests sleep. fur" }
-{ "l_orderkey": 1250, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, i" }
-{ "l_orderkey": 3811, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 31570.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly final dolphins? quickly ironic frets" }
-{ "l_orderkey": 4389, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "at the final excuses hinder carefully a" }
-{ "l_orderkey": 4454, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "equests run." }
-{ "l_orderkey": 5348, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-02-02", "l_receiptdate": "1997-12-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y according to the carefully pending acco" }
-{ "l_orderkey": 5893, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1804.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-18", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-08-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages wake sly" }
-{ "l_orderkey": 418, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "regular, silent pinto" }
 { "l_orderkey": 2656, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts serve deposi" }
 { "l_orderkey": 2662, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5412.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-30", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "olites cajole quickly along the b" }
 { "l_orderkey": 3046, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 27962.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-30", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending somas alongside of the slyly iro" }
+{ "l_orderkey": 3362, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2706.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-26", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its cajole blithely excuses. de" }
+{ "l_orderkey": 3650, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-07-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re about the pinto " }
+{ "l_orderkey": 3654, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts doze bravely ab" }
+{ "l_orderkey": 3811, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 31570.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly final dolphins? quickly ironic frets" }
 { "l_orderkey": 4001, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35178.0, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-13", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-06-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " dogged excuses. blithe" }
+{ "l_orderkey": 4032, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24354.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously according to" }
 { "l_orderkey": 4135, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "posits cajole furiously carefully" }
+{ "l_orderkey": 4387, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s hinder quietly across the pla" }
+{ "l_orderkey": 4389, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "at the final excuses hinder carefully a" }
+{ "l_orderkey": 4454, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "equests run." }
+{ "l_orderkey": 5090, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-03", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ular requests su" }
 { "l_orderkey": 5312, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38786.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-03-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly unusual" }
+{ "l_orderkey": 5348, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-02-02", "l_receiptdate": "1997-12-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y according to the carefully pending acco" }
+{ "l_orderkey": 5478, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42394.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " instructions; slyly even accounts hagg" }
 { "l_orderkey": 5699, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "kages. fin" }
+{ "l_orderkey": 5828, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25256.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special ideas haggle slyly ac" }
+{ "l_orderkey": 5862, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26158.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e fluffily. furiously" }
+{ "l_orderkey": 5893, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1804.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-18", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-08-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages wake sly" }
 { "l_orderkey": 5957, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15334.0, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final, pending packages" }
 { "l_orderkey": 35, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ", regular tithe" }
+{ "l_orderkey": 134, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. quickly regular" }
 { "l_orderkey": 321, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hockey players sleep slyly sl" }
+{ "l_orderkey": 548, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ideas. special accounts above the furiou" }
+{ "l_orderkey": 640, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36040.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oach according to the bol" }
+{ "l_orderkey": 807, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17119.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns haggle quickly across the furi" }
+{ "l_orderkey": 1122, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "t theodolites sleep. even, ironic" }
+{ "l_orderkey": 1154, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31535.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-30", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the carefully regular pinto beans boost" }
+{ "l_orderkey": 1287, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-08-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar packages. even, even" }
+{ "l_orderkey": 1472, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic theodolites hinder slyly slyly r" }
 { "l_orderkey": 1668, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic requests. bold, final ideas a" }
+{ "l_orderkey": 1761, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ons boost fu" }
+{ "l_orderkey": 2117, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 24327.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the carefully ironic ideas" }
 { "l_orderkey": 2374, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-26", "l_commitdate": "1993-12-15", "l_receiptdate": "1993-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending d" }
+{ "l_orderkey": 2528, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9010.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ely. fluffily even re" }
+{ "l_orderkey": 2534, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ideas. deposits use. slyly regular pa" }
 { "l_orderkey": 2726, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-04", "l_commitdate": "1993-01-29", "l_receiptdate": "1993-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously bold theodolites" }
 { "l_orderkey": 2883, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-26", "l_commitdate": "1995-03-04", "l_receiptdate": "1995-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s. final i" }
 { "l_orderkey": 2885, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40545.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-10-30", "l_receiptdate": "1993-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ess ideas. regular, silen" }
-{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lites sleep" }
-{ "l_orderkey": 5634, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 901.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ctions haggle carefully. carefully clo" }
-{ "l_orderkey": 807, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17119.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns haggle quickly across the furi" }
-{ "l_orderkey": 1761, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ons boost fu" }
-{ "l_orderkey": 3843, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake. slyly even packages boost " }
-{ "l_orderkey": 4355, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ought to mold. blithely pending ideas " }
-{ "l_orderkey": 5121, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1802.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-06-28", "l_receiptdate": "1992-08-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " final, regular account" }
-{ "l_orderkey": 5409, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8109.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-15", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " unusual, unusual reques" }
-{ "l_orderkey": 5760, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng the acco" }
-{ "l_orderkey": 5984, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7208.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its. express," }
-{ "l_orderkey": 640, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36040.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oach according to the bol" }
-{ "l_orderkey": 1122, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "t theodolites sleep. even, ironic" }
-{ "l_orderkey": 1154, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31535.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-30", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the carefully regular pinto beans boost" }
-{ "l_orderkey": 1472, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic theodolites hinder slyly slyly r" }
 { "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-10", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "the quickly even dolph" }
+{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lites sleep" }
 { "l_orderkey": 3457, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tructions haggle alongsid" }
+{ "l_orderkey": 3843, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake. slyly even packages boost " }
+{ "l_orderkey": 3940, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thily. deposits cajole." }
 { "l_orderkey": 4102, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-14", "l_commitdate": "1996-04-29", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even requests; regular pinto" }
 { "l_orderkey": 4293, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30634.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ions sleep blithely on" }
 { "l_orderkey": 4323, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-04", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "the slyly bold deposits slee" }
+{ "l_orderkey": 4355, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ought to mold. blithely pending ideas " }
 { "l_orderkey": 4452, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42347.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. slyly regular cour" }
-{ "l_orderkey": 134, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. quickly regular" }
-{ "l_orderkey": 548, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ideas. special accounts above the furiou" }
-{ "l_orderkey": 1287, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-08-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar packages. even, even" }
-{ "l_orderkey": 2117, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 24327.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the carefully ironic ideas" }
-{ "l_orderkey": 2528, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9010.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ely. fluffily even re" }
-{ "l_orderkey": 2534, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ideas. deposits use. slyly regular pa" }
-{ "l_orderkey": 3940, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thily. deposits cajole." }
 { "l_orderkey": 4580, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-13", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "requests. quickly silent asymptotes sle" }
+{ "l_orderkey": 5121, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1802.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-06-28", "l_receiptdate": "1992-08-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " final, regular account" }
+{ "l_orderkey": 5409, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8109.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-15", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " unusual, unusual reques" }
+{ "l_orderkey": 5634, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 901.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ctions haggle carefully. carefully clo" }
+{ "l_orderkey": 5760, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng the acco" }
+{ "l_orderkey": 5984, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7208.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its. express," }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
index 1dc730f..edea622 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
@@ -51,4 +51,9 @@
     Field=l_linenumber
   ]
   ASC
+  FieldAccessor [
+    Variable [ Name=$l ]
+    Field=l_orderkey
+  ]
+  ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
index 87320ab..0460dc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
@@ -14,4 +14,14 @@
     Field=l_partkey
   ]
   DESC
+  FieldAccessor [
+    Variable [ Name=$i ]
+    Field=l_orderkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$i ]
+    Field=l_linenumber
+  ]
+  ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
index 36fb0a1..9ea33d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
@@ -137,4 +137,14 @@
     ]
   ]
   ASC
+  Variable [ Name=$itv ]
+  ASC
+  FunctionCall test.get-overlapping-interval@2[
+    FieldAccessor [
+      Variable [ Name=$gen0 ]
+      Field=bin
+    ]
+    Variable [ Name=$itv ]
+  ]
+  ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 3236e62..f235d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4339,6 +4339,16 @@
         <output-dir compare="Text">uuid</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_seq_merge">
+        <output-dir compare="Text">p_sort_seq_merge</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_num_samples">
+        <output-dir compare="Text">p_sort_num_samples</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index">
     <test-group name="index/validations">
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 6d89ff5..4c58ad7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -22,6 +22,7 @@
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
 import org.apache.hyracks.api.config.Section;
@@ -58,7 +59,12 @@
                         + "other integer values dictate the number of query execution parallel partitions. The system will "
                         + "fall back to use the number of all available CPU cores in the cluster as the degree of parallelism "
                         + "if the number set by a user is too large or too small"),
-        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)");
+        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
+        COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
+        COMPILER_SORT_SAMPLES(
+                INTEGER,
+                AlgebricksConfig.SORT_SAMPLES,
+                "The number of samples parallel sorting should " + "take from each partition");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -106,6 +112,10 @@
 
     public static final String COMPILER_PARALLELISM_KEY = Option.COMPILER_PARALLELISM.ini();
 
+    public static final String COMPILER_SORT_PARALLEL_KEY = Option.COMPILER_SORT_PARALLEL.ini();
+
+    public static final String COMPILER_SORT_SAMPLES_KEY = Option.COMPILER_SORT_SAMPLES.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -140,4 +150,13 @@
         int value = accessor.getInt(Option.COMPILER_STRINGOFFSET);
         return value > 0 ? 1 : 0;
     }
+
+    public boolean getSortParallel() {
+        return accessor.getBoolean(Option.COMPILER_SORT_PARALLEL);
+    }
+
+    public int getSortSamples() {
+        int numSamples = accessor.getInt(Option.COMPILER_SORT_SAMPLES);
+        return numSamples > 0 ? numSamples : AlgebricksConfig.SORT_SAMPLES;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index dda6f7b..1bf2447 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -179,6 +179,7 @@
     public static final int COMPILATION_TYPE_MISMATCH_GENERIC = 1091;
     public static final int ILLEGAL_SET_PARAMETER = 1092;
     public static final int COMPILATION_TRANSLATION_ERROR = 1093;
+    public static final int RANGE_MAP_ERROR = 1094;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 4c8c866..5629b97 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -166,6 +166,7 @@
 1090 = Field %1$s must be of an array of type %2$s but found to contain an item of type %3$s
 1092 = Parameter %1$s cannot be set
 1093 = A parser error has occurred. The detail exception: %1$s
+1094 = Cannot parse range map: %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-aql/pom.xml b/asterixdb/asterix-lang-aql/pom.xml
index 6181fe6..f202192 100644
--- a/asterixdb/asterix-lang-aql/pom.xml
+++ b/asterixdb/asterix-lang-aql/pom.xml
@@ -158,10 +158,6 @@
       <artifactId>commons-io</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-dataflow-common</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-om</artifactId>
       <version>${project.version}</version>
@@ -184,10 +180,6 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-data-std</artifactId>
-    </dependency>
   </dependencies>
 
 </project>
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 856073d..35c2ae8 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -67,12 +67,12 @@
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
 import org.apache.asterix.lang.aql.expression.UnionExpr;
-import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.aql.util.AQLFormatPrintUtil;
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -150,6 +150,7 @@
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -212,6 +213,10 @@
         return s.substring(1).trim();
     }
 
+    private static IParser createNewParser(String statement) {
+        return new AQLParser(statement);
+    }
+
     private static void checkBindingVariable(Expression returnExpression, VariableExpr var,
                                              ILangExpression bodyExpression) throws ParseException {
        if (returnExpression != null && var == null) {
@@ -2419,7 +2424,7 @@
           }
           if (hint.startsWith(RANGE_HINT)) {
             try{
-              oc.setRangeMap(RangeMapBuilder.parseHint(hint.substring(RANGE_HINT.length())));
+              oc.setRangeMap(RangeMapBuilder.parseHint(createNewParser(hint.substring(RANGE_HINT.length()))));
             } catch (CompilationException e) {
               throw new ParseException(e.getMessage());
             }
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index 7d68b8b..dde41e0 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -103,5 +103,9 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-data-std</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index faefe1d..af8c725 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -25,12 +25,12 @@
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class OrderbyClause extends AbstractClause {
     private List<Expression> orderbyList;
     private List<OrderModifier> modifierList;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap; // can be null
     private int numFrames = -1;
     private int numTuples = -1;
 
@@ -90,17 +90,17 @@
         this.numTuples = numTuples;
     }
 
-    public IRangeMap getRangeMap() {
+    public RangeMap getRangeMap() {
         return rangeMap;
     }
 
-    public void setRangeMap(IRangeMap rangeMap) {
+    public void setRangeMap(RangeMap rangeMap) {
         this.rangeMap = rangeMap;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(modifierList, numFrames, numTuples, orderbyList);
+        return Objects.hash(modifierList, numFrames, numTuples, orderbyList, rangeMap);
     }
 
     @Override
@@ -113,6 +113,7 @@
         }
         OrderbyClause target = (OrderbyClause) object;
         return Objects.equals(modifierList, target.modifierList) && numFrames == target.numFrames
-                && numTuples == target.numTuples && orderbyList.equals(target.orderbyList);
+                && numTuples == target.numTuples && orderbyList.equals(target.orderbyList)
+                && Objects.equals(rangeMap, target.rangeMap);
     }
 }
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
similarity index 89%
rename from asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
rename to asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
index 23b0066..c505c1c 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
@@ -16,19 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.lang.aql.util;
+package org.apache.asterix.lang.common.util;
 
 import java.io.DataOutput;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.ListConstructor;
@@ -52,18 +51,18 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
-public abstract class RangeMapBuilder {
-    private static final IParserFactory parserFactory = new AQLParserFactory();
+public class RangeMapBuilder {
 
-    public static IRangeMap parseHint(Object hint) throws CompilationException {
+    private RangeMapBuilder() {
+    }
+
+    public static RangeMap parseHint(IParser parser) throws CompilationException {
         ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
         DataOutput out = abvs.getDataOutput();
         abvs.reset();
 
-        IParser parser = parserFactory.createParser((String) hint);
         List<Statement> hintStatements = parser.parse();
         if (hintStatements.size() != 1) {
             throw new CompilationException("Only one range statement is allowed for the range hint.");
@@ -79,7 +78,7 @@
             throw new CompilationException("The range hint must be a list.");
         }
         List<Expression> el = ((ListConstructor) q.getBody()).getExprList();
-        int offsets[] = new int[el.size()];
+        int[] offsets = new int[el.size()];
 
         // Loop over list of literals
         for (int i = 0; i < el.size(); ++i) {
@@ -142,11 +141,11 @@
                             + item.getKind() + " type of expressions.");
             }
         } catch (HyracksDataException e) {
-            throw new CompilationException(e.getMessage());
+            throw new CompilationException(ErrorCode.RANGE_MAP_ERROR, e, item.getSourceLocation(), e.getMessage());
         }
     }
 
-    public static void verifyRangeOrder(IRangeMap rangeMap, boolean ascending) throws CompilationException {
+    public static void verifyRangeOrder(RangeMap rangeMap, boolean ascending) throws CompilationException {
         // TODO Add support for composite fields.
         int fieldIndex = 0;
         int fieldType = rangeMap.getTag(0, 0);
@@ -162,9 +161,8 @@
             }
             int previousSplit = split - 1;
             try {
-                c = comparator.compare(rangeMap.getByteArray(fieldIndex, previousSplit),
-                        rangeMap.getStartOffset(fieldIndex, previousSplit),
-                        rangeMap.getLength(fieldIndex, previousSplit), rangeMap.getByteArray(fieldIndex, split),
+                c = comparator.compare(rangeMap.getByteArray(), rangeMap.getStartOffset(fieldIndex, previousSplit),
+                        rangeMap.getLength(fieldIndex, previousSplit), rangeMap.getByteArray(),
                         rangeMap.getStartOffset(fieldIndex, split), rangeMap.getLength(fieldIndex, split));
             } catch (HyracksDataException e) {
                 throw new CompilationException(e);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index e2a8759..13d1f8d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -73,6 +73,7 @@
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -149,6 +150,7 @@
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
 import org.apache.asterix.lang.sqlpp.clause.FromTerm;
@@ -200,7 +202,6 @@
     private static final String SKIP_SECONDARY_INDEX_SEARCH_HINT = "skip-index";
     private static final String VAL_FILE_HINT = "val-files";
     private static final String VAL_FILE_SAME_INDEX_HINT = "val-file-same-idx";
-
     private static final String GEN_FIELDS_HINT = "gen-fields";
 
     // data generator hints
@@ -241,6 +242,10 @@
         return s.substring(1).trim();
     }
 
+    private static IParser createNewParser(String statement) {
+        return new SQLPPParser(statement);
+    }
+
     private Token getHintToken(Token t) {
         return t.specialToken;
     }
@@ -3062,6 +3067,13 @@
             oc.setNumFrames(numFrames);
             oc.setNumTuples(numTuples);
           }
+          if (hint.startsWith(RANGE_HINT)) {
+            try {
+              oc.setRangeMap(RangeMapBuilder.parseHint(createNewParser(hint.substring(RANGE_HINT.length()))));
+            } catch (CompilationException e) {
+              throw new SqlppParseException(getSourceLocation(getHintToken(token)), e.getMessage());
+            }
+          }
         }
       }
     <BY> orderbyExpr = Expression()
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 910c159..1ff6e57 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -103,6 +103,7 @@
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAPointTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAStringTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ListOfSamplesTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
@@ -477,6 +478,10 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-stddev", 1);
     public static final FunctionIdentifier LOCAL_STDDEV =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-stddev", 1);
+    public static final FunctionIdentifier LOCAL_SAMPLING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sampling", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RANGE_MAP =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-range-map", 1);
 
     public static final FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
     public static final FunctionIdentifier SCALAR_COUNT =
@@ -1381,6 +1386,8 @@
         addPrivateFunction(LOCAL_STDDEV, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addFunction(STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
+        addPrivateFunction(LOCAL_SAMPLING, ListOfSamplesTypeComputer.INSTANCE, true);
+        addPrivateFunction(RANGE_MAP, ABinaryTypeComputer.INSTANCE, true);
 
         addPrivateFunction(SERIAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
@@ -1832,6 +1839,14 @@
 
         addScalarAgg(FIRST_ELEMENT, SCALAR_FIRST_ELEMENT);
 
+        // RANGE_MAP
+        addAgg(RANGE_MAP);
+        addAgg(LOCAL_SAMPLING);
+        addLocalAgg(RANGE_MAP, LOCAL_SAMPLING);
+        addIntermediateAgg(LOCAL_SAMPLING, RANGE_MAP);
+        addIntermediateAgg(RANGE_MAP, RANGE_MAP);
+        addGlobalAgg(RANGE_MAP, RANGE_MAP);
+
         // MIN
 
         addAgg(MIN);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java
new file mode 100644
index 0000000..1ae72e4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+/**
+ * List of samples type: [[ANY], [ANY],...]. Each inner list constitutes one sample. Inside the inner list (the sample),
+ * each item (or field) has its type tag.
+ */
+public class ListOfSamplesTypeComputer extends AbstractResultTypeComputer {
+
+    public static final AOrderedListType TYPE =
+            new AOrderedListType(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE, null);
+    public static final ListOfSamplesTypeComputer INSTANCE = new ListOfSamplesTypeComputer();
+
+    private ListOfSamplesTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return TYPE;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java
new file mode 100644
index 0000000..55d381d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.ListOfSamplesTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class LocalSamplingAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private int numSamples;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new LocalSamplingAggregateDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_NUM_SAMPLES;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.LOCAL_SAMPLING;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        numSamples = (int) states[0];
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new LocalSamplingAggregateFunction(args, ctx, numSamples);
+            }
+        };
+    }
+
+    private class LocalSamplingAggregateFunction implements IAggregateEvaluator {
+        private final int numSamplesRequired;
+        private final ArrayBackedValueStorage storage;
+        private final IAsterixListBuilder listOfSamplesBuilder;
+        private final IAsterixListBuilder oneSampleBuilder;
+        private final IScalarEvaluator[] sampledFieldsEval;
+        private final IPointable inputFieldValue;
+        private int numSamplesTaken;
+
+        /**
+         * @param args the fields that constitute a sample, e.g., $$1, $$2
+         * @param context Hyracks task
+         * @throws HyracksDataException
+         */
+        private LocalSamplingAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+                int numSamplesRequired) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            inputFieldValue = new VoidPointable();
+            sampledFieldsEval = new IScalarEvaluator[args.length];
+            for (int i = 0; i < args.length; i++) {
+                sampledFieldsEval[i] = args[i].createScalarEvaluator(context);
+            }
+            oneSampleBuilder = new OrderedListBuilder();
+            listOfSamplesBuilder = new OrderedListBuilder();
+            listOfSamplesBuilder.reset(ListOfSamplesTypeComputer.TYPE);
+            this.numSamplesRequired = numSamplesRequired > 0 ? numSamplesRequired
+                    : (int) CompilerProperties.Option.COMPILER_SORT_SAMPLES.defaultValue();
+        }
+
+        @Override
+        public void init() throws HyracksDataException {
+            numSamplesTaken = 0;
+            listOfSamplesBuilder.reset(ListOfSamplesTypeComputer.TYPE);
+        }
+
+        /**
+         * Receives data stream one tuple at a time from a data source and records samples.
+         * @param tuple one sample
+         * @throws HyracksDataException
+         */
+        @Override
+        public void step(IFrameTupleReference tuple) throws HyracksDataException {
+            if (numSamplesTaken >= numSamplesRequired) {
+                return;
+            }
+            // start over for a new sample
+            oneSampleBuilder.reset((AbstractCollectionType) ListOfSamplesTypeComputer.TYPE.getItemType());
+
+            for (IScalarEvaluator fieldEval : sampledFieldsEval) {
+                // add fields to make up one sample
+                fieldEval.evaluate(tuple, inputFieldValue);
+                oneSampleBuilder.addItem(inputFieldValue);
+            }
+            // prepare the sample to add it to the list of samples
+            storage.reset();
+            oneSampleBuilder.write(storage.getDataOutput(), true);
+            listOfSamplesBuilder.addItem(storage);
+            numSamplesTaken++;
+        }
+
+        /**
+         * Sends the list of samples to the global range-map generator.
+         * @param result list of samples
+         * @throws HyracksDataException
+         */
+        @Override
+        public void finish(IPointable result) throws HyracksDataException {
+            storage.reset();
+            if (numSamplesTaken == 0) {
+                // empty partition? then send system null as an indication of empty partition.
+                try {
+                    storage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+                    result.set(storage);
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+            } else {
+                listOfSamplesBuilder.write(storage.getDataOutput(), true);
+                result.set(storage);
+            }
+        }
+
+        @Override
+        public void finishPartial(IPointable result) throws HyracksDataException {
+            finish(result);
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java
new file mode 100644
index 0000000..c967a94
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+/**
+ * <pre>
+ * {@see {@link org.apache.hyracks.dataflow.common.data.partition.range.RangeMap}} for some description of the range map
+ * structure that is produced by this function. Given a list of samples and a number of partitions "k", the algorithm
+ * of this function operates as follows (s = sample):
+ * It picks (k - 1) split points out of the samples by dividing num_samples/num_partitions. For 4 partitions, it's 3:
+ * s0,s1,s2,s3,s4,s5,s6,s7,s8,s9,s10,s11,s12,s13,s14,s15; 16/4 = 4; range map = [s3, s7, s11]
+ *           |           |             |
+ *
+ * s0,s1,s2,s3,s4,s5,s6; 7/4 = 2; range map = [s1, s3, s5]
+ *     |     |     |
+ *
+ * s0,s1,s2,s3,s4; 5/4 = 2; range map = [s1, s3, s4]; if we go out of bound for the last split, we pick the last item.
+ *     |     |  |
+ *
+ * s0,s1,s2,s3; if #_samples <= #_partitions, we sweep from the beginning (should be rare). range map = [s0, s1, s2]
+ *  |  |  |
+ *
+ * s0,s1; if there are way less samples, we sweep and repeat the last item; range map = [s0, s1, s1];
+ * Note: a sample (and therefore also a split point) could be single-column or multi-column.
+ * </pre>
+ */
+public class RangeMapAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private boolean[] ascendingFlags;
+    private int numOfPartitions;
+    private int numOrderFields;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RangeMapAggregateDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_SORTING_PARAMETERS;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANGE_MAP;
+    }
+
+    /**
+     * The sampling function, which generates the splitting vector, needs to know the number of partitions in order to
+     * determine how many split points to pick out of the samples. It also needs to know the ascending/descending of
+     * each sort field so that it can sort the samples accordingly first and then pick the (number of partitions - 1)
+     * split points out of the sorted samples.
+     * @param states states[0]: number of partitions, states[1]: ascending flags
+     */
+    @Override
+    public void setImmutableStates(Object... states) {
+        numOfPartitions = (int) states[0];
+        ascendingFlags = (boolean[]) states[1];
+        numOrderFields = ascendingFlags.length;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new GlobalSamplingAggregateFunction(args, ctx, ascendingFlags, numOfPartitions, numOrderFields);
+            }
+        };
+    }
+
+    private class GlobalSamplingAggregateFunction implements IAggregateEvaluator {
+        private final IScalarEvaluator localSamplesEval;
+        private final IPointable localSamples;
+        private final List<List<byte[]>> finalSamples;
+        private final Comparator<List<byte[]>> comparator;
+        private final int numOfPartitions;
+        private final int numOrderByFields;
+        private final ListAccessor listOfSamples;
+        private final ListAccessor oneSample;
+        private final IPointable oneSamplePointable;
+        private final ArrayBackedValueStorage oneSampleStorage;
+        private final IPointable field;
+        private final ArrayBackedValueStorage storage;
+
+        @SuppressWarnings("unchecked")
+        private GlobalSamplingAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+                boolean[] ascending, int numOfPartitions, int numOrderByFields) throws HyracksDataException {
+            localSamples = new VoidPointable();
+            localSamplesEval = args[0].createScalarEvaluator(context);
+            finalSamples = new ArrayList<>();
+            comparator = createComparator(ascending);
+            this.numOfPartitions = numOfPartitions;
+            this.numOrderByFields = numOrderByFields;
+            listOfSamples = new ListAccessor();
+            oneSample = new ListAccessor();
+            oneSamplePointable = new VoidPointable();
+            oneSampleStorage = new ArrayBackedValueStorage();
+            field = new VoidPointable();
+            storage = new ArrayBackedValueStorage();
+        }
+
+        @Override
+        public void init() throws HyracksDataException {
+            finalSamples.clear();
+        }
+
+        /**
+         * Receives the local samples and appends them to the final list of samples.
+         * @param tuple the partition's samples
+         * @throws HyracksDataException
+         */
+        @Override
+        public void step(IFrameTupleReference tuple) throws HyracksDataException {
+            // check if empty stream (system_null), i.e. partition is empty, so no samples
+            localSamplesEval.evaluate(tuple, localSamples);
+            byte tag = localSamples.getByteArray()[localSamples.getStartOffset()];
+            if (tag == ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG) {
+                return;
+            }
+            // deserialize the samples received from the local partition
+            listOfSamples.reset(localSamples.getByteArray(), localSamples.getStartOffset());
+            int numberOfSamples = listOfSamples.size();
+
+            // "sample" & "addedSample" are lists to support multi-column instead of one value, i.e. <3,"dept">
+            List<byte[]> addedSample;
+            int numberOfFields;
+            // add the samples to the final samples
+            try {
+                for (int i = 0; i < numberOfSamples; i++) {
+                    oneSampleStorage.reset();
+                    listOfSamples.getOrWriteItem(i, oneSamplePointable, oneSampleStorage);
+                    oneSample.reset(oneSamplePointable.getByteArray(), oneSamplePointable.getStartOffset());
+                    numberOfFields = oneSample.size();
+                    addedSample = new ArrayList<>(numberOfFields);
+                    for (int j = 0; j < numberOfFields; j++) {
+                        storage.reset();
+                        oneSample.getOrWriteItem(j, field, storage);
+                        addedSample.add(Arrays.copyOfRange(field.getByteArray(), field.getStartOffset(),
+                                field.getStartOffset() + field.getLength()));
+                    }
+                    finalSamples.add(addedSample);
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+
+        /**
+         * Produces the range map out of the collected samples from each partition. The final list of samples is sorted
+         * first. Then, we select the split points by dividing the samples evenly.
+         * @param result contains the serialized range map.
+         * @throws HyracksDataException
+         */
+        @Override
+        public void finish(IPointable result) throws HyracksDataException {
+            // storage == all serialized split values of all split points
+            storage.reset();
+            DataOutput allSplitValuesOut = storage.getDataOutput();
+            int[] endOffsets;
+            try {
+                // check if empty dataset, i.e. no samples have been received from any partition
+                if (finalSamples.isEmpty()) {
+                    // a range map with null values
+                    endOffsets = new int[numOrderByFields];
+                    for (int sortField = 0; sortField < numOrderByFields; sortField++) {
+                        allSplitValuesOut.write(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                        endOffsets[sortField] = storage.getLength();
+                    }
+                } else {
+                    finalSamples.sort(comparator);
+                    // divide the samples evenly and pick the boundaries as split points
+                    int nextSplitOffset = (int) Math.ceil(finalSamples.size() / (double) numOfPartitions);
+                    int nextSplitIndex = nextSplitOffset - 1;
+                    int endOffsetsCounter = 0;
+                    int numRequiredSplits = numOfPartitions - 1;
+                    endOffsets = new int[numRequiredSplits * numOrderByFields];
+                    List<byte[]> sample;
+                    for (int split = 1; split <= numRequiredSplits; split++) {
+                        // pick the split point from sorted samples (could be <3> or <4,"John"> if it's multi-column)
+                        sample = finalSamples.get(nextSplitIndex);
+                        for (int column = 0; column < sample.size(); column++) {
+                            allSplitValuesOut.write(sample.get(column));
+                            endOffsets[endOffsetsCounter++] = storage.getLength();
+                        }
+                        // go to the next split point
+                        nextSplitIndex += nextSplitOffset;
+                        // in case we go beyond the boundary of samples, we pick the last sample repeatedly
+                        if (nextSplitIndex >= finalSamples.size()) {
+                            nextSplitIndex = finalSamples.size() - 1;
+                        }
+                    }
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+
+            serializeRangemap(numOrderByFields, storage.getByteArray(), endOffsets, result);
+        }
+
+        @Override
+        public void finishPartial(IPointable result) throws HyracksDataException {
+            finish(result);
+        }
+
+        /**
+         * Creates the comparator that sorts all the collected samples before picking split points.
+         * @param ascending ascending or descending flag for each sort column.
+         * @return the described comparator
+         */
+        private Comparator<List<byte[]>> createComparator(boolean[] ascending) {
+            // create the generic comparator for each sort field
+            IBinaryComparator[] fieldsComparators = new IBinaryComparator[ascending.length];
+            for (int i = 0; i < ascending.length; i++) {
+                if (ascending[i]) {
+                    fieldsComparators[i] = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+                } else {
+                    fieldsComparators[i] = AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+                }
+            }
+
+            return (splitPoint1, splitPoint2) -> {
+                try {
+                    // two split points must have the same num of fields
+                    int numFields = splitPoint1.size();
+                    int result = 0;
+                    byte[] field1;
+                    byte[] field2;
+                    for (int i = 0; i < numFields; i++) {
+                        field1 = splitPoint1.get(i);
+                        field2 = splitPoint2.get(i);
+                        result = fieldsComparators[i].compare(field1, 0, field1.length, field2, 0, field2.length);
+                        if (result != 0) {
+                            return result;
+                        }
+                    }
+                    return result;
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+            };
+        }
+
+        /**
+         * Serializes the range map object defined by the below attributes into the "result". The range map object is
+         * serialized as binary data.
+         * @param numberFields the number of order-by fields (the sort fields)
+         * @param splitValues the serialized split values stored one after the other
+         * @param endOffsets the end offsets of each split value
+         * @param result where the range map object is serialized
+         * @throws HyracksDataException
+         */
+        private void serializeRangemap(int numberFields, byte[] splitValues, int[] endOffsets, IPointable result)
+                throws HyracksDataException {
+            ArrayBackedValueStorage serRangeMap = new ArrayBackedValueStorage();
+            IntegerSerializerDeserializer.write(numberFields, serRangeMap.getDataOutput());
+            ByteArraySerializerDeserializer.write(splitValues, serRangeMap.getDataOutput());
+            IntArraySerializerDeserializer.write(endOffsets, serRangeMap.getDataOutput());
+
+            result.set(serRangeMap.getByteArray(), serRangeMap.getStartOffset(), serRangeMap.getLength());
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 9541d69..5affbca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -68,6 +68,7 @@
 import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.RangeMapAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalStddevAggregateDescriptor;
@@ -78,6 +79,7 @@
 import org.apache.asterix.runtime.aggregates.std.LocalAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalMaxAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSamplingAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlMaxAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlMinAggregateDescriptor;
@@ -458,6 +460,8 @@
         fc.add(LocalStddevAggregateDescriptor.FACTORY);
         fc.add(IntermediateStddevAggregateDescriptor.FACTORY);
         fc.add(GlobalStddevAggregateDescriptor.FACTORY);
+        fc.add(LocalSamplingAggregateDescriptor.FACTORY);
+        fc.add(RangeMapAggregateDescriptor.FACTORY);
 
         // serializable aggregates
         fc.add(SerializableCountAggregateDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index b9c58c7..44e3eb7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -19,6 +19,9 @@
 
 package org.apache.asterix.runtime.functions;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
@@ -42,9 +45,6 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * Implementations of {@link IFunctionTypeInferer} for built-in functions
  */
@@ -95,6 +95,26 @@
         }
     };
 
+    public static final IFunctionTypeInferer SET_SORTING_PARAMETERS = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression funCallExpr = (AbstractFunctionCallExpression) expr;
+            Object[] sortingParameters = funCallExpr.getOpaqueParameters();
+            fd.setImmutableStates(sortingParameters[0], sortingParameters[1]);
+        }
+    };
+
+    public static final IFunctionTypeInferer SET_NUM_SAMPLES = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression funCallExpr = (AbstractFunctionCallExpression) expr;
+            Object[] samplingParameters = funCallExpr.getOpaqueParameters();
+            fd.setImmutableStates(samplingParameters[0]);
+        }
+    };
+
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
         public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 2d13baf..3794328 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -28,6 +28,7 @@
     EXCHANGE,
     DELEGATE_OPERATOR,
     EXTERNAL_LOOKUP,
+    FORWARD,
     GROUP,
     INDEX_INSERT_DELETE_UPSERT,
     INNERJOIN,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
index fc702ce..cbe0882 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
@@ -22,7 +22,8 @@
     // hints
     public static final String USE_HASH_GROUP_BY = "USE_HASH_GROUP_BY"; // -->
     public static final String USE_EXTERNAL_GROUP_BY = "USE_EXTERNAL_GROUP_BY"; // -->
-    public static final String USE_RANGE_CONNECTOR = "USE_RANGE_CONNECTOR"; // -->
+    public static final String USE_STATIC_RANGE = "USE_STATIC_RANGE"; // -->
+    public static final String USE_DYNAMIC_RANGE = "USE_DYNAMIC_RANGE";
     // Boolean
     public static final String CARDINALITY = "CARDINALITY"; // -->
     // Integer
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index db9728b..ac1de5a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -30,6 +30,7 @@
     DELEGATE_OPERATOR,
     EXTERNAL_GROUP_BY,
     EXTERNAL_LOOKUP,
+    FORWARD,
     HASH_GROUP_BY,
     HASH_PARTITION_EXCHANGE,
     HASH_PARTITION_MERGE_EXCHANGE,
@@ -54,6 +55,7 @@
     RANDOM_MERGE_EXCHANGE,
     RANGE_PARTITION_EXCHANGE,
     RANGE_PARTITION_MERGE_EXCHANGE,
+    SEQUENTIAL_MERGE_EXCHANGE,
     REPLICATE,
     RTREE_SEARCH,
     RUNNING_AGGREGATE,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
index 80d6f95..1cc206f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -205,19 +206,7 @@
                     return false;
                 }
             }
-            if (opaqueParameters != null) {
-                if (opaqueParameters.length != fce.opaqueParameters.length) {
-                    return false;
-                }
-                for (int i = 0; i < opaqueParameters.length; i++) {
-                    Object opaqueParameter = opaqueParameters[i];
-                    Object fceOpaqueParameter = fce.opaqueParameters[i];
-                    if (!opaqueParameter.equals(fceOpaqueParameter)) {
-                        return false;
-                    }
-                }
-            }
-            return true;
+            return Arrays.deepEquals(opaqueParameters, fce.opaqueParameters);
         }
     }
 
@@ -228,9 +217,7 @@
             h = h * 41 + e.getValue().hashCode();
         }
         if (opaqueParameters != null) {
-            for (int i = 0; i < opaqueParameters.length; i++) {
-                h = h * 31 + opaqueParameters[i].hashCode();
-            }
+            h = h * 31 + Arrays.deepHashCode(opaqueParameters);
         }
         return h;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
index bdd820e..97c4252 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
@@ -60,6 +60,8 @@
         fun.setStepTwoAggregate(stepTwoAggregate);
         fun.setStepOneAggregate(stepOneAggregate);
         fun.setSourceLocation(sourceLoc);
+        // opaqueParameters are not really cloned
+        fun.setOpaqueParameters(getOpaqueParameters());
         return fun;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
index 3bb0f47..62b6a2d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
@@ -23,6 +23,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
@@ -39,7 +40,7 @@
 public abstract class AbstractReplicateOperator extends AbstractLogicalOperator {
 
     private int outputArity;
-    protected boolean[] outputMaterializationFlags;
+    private boolean[] outputMaterializationFlags;
     private List<Mutable<ILogicalOperator>> outputs;
 
     public AbstractReplicateOperator(int outputArity) {
@@ -95,6 +96,19 @@
         return outputs;
     }
 
+    public void setOutputs(List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs) {
+        // shrinking or expanding num of outputs
+        if (outputMaterializationFlags.length != newOutputs.size()) {
+            outputMaterializationFlags = new boolean[newOutputs.size()];
+        }
+        outputs.clear();
+        for (int i = 0; i < newOutputs.size(); i++) {
+            outputs.add(newOutputs.get(i).first);
+            outputMaterializationFlags[i] = newOutputs.get(i).second;
+        }
+        outputArity = newOutputs.size();
+    }
+
     @Override
     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
         return createPropagatingAllInputsTypeEnvironment(ctx);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java
new file mode 100644
index 0000000..db11712
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.logical;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * Forward operator is used to forward data to different NCs based on a range map that is computed dynamically
+ * by doing a pass over the data itself to infer the range map. The operator takes two inputs:
+ * 1. Tuples/data (at index 0). The data is forwarded to the range-based connector which routes it to the target NC.
+ * 2. Range map (at index 1). The range map will be stored in Hyracks context, and the connector will pick it up.
+ * Forward operator will receive the range map when it is broadcast by the operator generating the range map after which
+ * the forward operator will start forwarding the data.
+ */
+public class ForwardOperator extends AbstractLogicalOperator {
+
+    private final String rangeMapKey;
+    private final Mutable<ILogicalExpression> rangeMapExpression;
+
+    public ForwardOperator(String rangeMapKey, Mutable<ILogicalExpression> rangeMapExpression) {
+        super();
+        this.rangeMapKey = rangeMapKey;
+        this.rangeMapExpression = rangeMapExpression;
+    }
+
+    public String getRangeMapKey() {
+        return rangeMapKey;
+    }
+
+    public Mutable<ILogicalExpression> getRangeMapExpression() {
+        return rangeMapExpression;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.FORWARD;
+    }
+
+    @Override
+    public void recomputeSchema() throws AlgebricksException {
+        // schema is equal to the schema of the data source at idx 0
+        setSchema(inputs.get(0).getValue().getSchema());
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        return visitor.transform(rangeMapExpression);
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitForwardOperator(this, arg);
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return new VariablePropagationPolicy() {
+
+            @Override
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+                    throws AlgebricksException {
+                // propagate the variables of the data source at idx 0
+                if (sources.length > 0) {
+                    target.addAllVariables(sources[0]);
+                }
+            }
+        };
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        // propagate the type environment of the data source at idx 0
+        ITypeEnvPointer[] envPointers = new ITypeEnvPointer[] { new OpRefTypeEnvPointer(inputs.get(0), ctx) };
+        return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMissableTypeComputer(),
+                ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index d0aea60..9d853eb 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -31,11 +31,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -276,6 +277,11 @@
     }
 
     @Override
+    public Long visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return op.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    @Override
     public Long visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
         long cardinality = UNKNOWN;
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index d0d121f..16fc1ed 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -53,6 +53,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -560,6 +561,12 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, IOptimizationContext ctx) throws AlgebricksException {
         setEmptyFDsEqClasses(op, ctx);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 90c0067..2b5e569 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -595,6 +596,18 @@
     }
 
     @Override
+    public Boolean visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        AbstractLogicalOperator argOperator = (AbstractLogicalOperator) arg;
+        if (argOperator.getOperatorTag() != LogicalOperatorTag.FORWARD) {
+            return Boolean.FALSE;
+        }
+        ForwardOperator otherOp = (ForwardOperator) copyAndSubstituteVar(op, arg);
+        ILogicalExpression rangeMapExp = op.getRangeMapExpression().getValue();
+        ILogicalExpression otherRangeMapExp = otherOp.getRangeMapExpression().getValue();
+        return rangeMapExp.equals(otherRangeMapExp) && op.getRangeMapKey().equals(otherOp.getRangeMapKey());
+    }
+
+    @Override
     public Boolean visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
         return true;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 2caa252..742d485 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -284,6 +285,12 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e0210cc..0196db6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -587,6 +588,14 @@
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        ForwardOperator opCopy = new ForwardOperator(op.getRangeMapKey(),
+                exprDeepCopyVisitor.deepCopyExpressionReference(op.getRangeMapExpression()));
+        deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
+        return opCopy;
+    }
+
+    @Override
     public ILogicalOperator visitDelegateOperator(DelegateOperator op, ILogicalOperator arg)
             throws AlgebricksException {
         throw new UnsupportedOperationException();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 6dfe254..7d3d676 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -30,11 +30,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -289,6 +290,12 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext arg) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 0db0f74..c6f0c14 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -342,6 +343,11 @@
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return new ForwardOperator(op.getRangeMapKey(), deepCopyExpressionRef(op.getRangeMapExpression()));
+    }
+
+    @Override
     public ILogicalOperator visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         return new SinkOperator();
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index c96276f..f36f604 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -32,11 +32,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -278,6 +279,11 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, IOptimizationContext arg)
             throws AlgebricksException {
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index ec96d48..5d0ef6a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -36,11 +36,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -278,6 +279,11 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 59ccd84..70ccf6d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -318,6 +319,13 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        // only consider variables from the branch of the data source
+        VariableUtilities.getLiveVariables(op.getInputs().get(0).getValue(), schemaVariables);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         standardLayout(op);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 3587e29..c62f555 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -460,6 +461,14 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        op.getRangeMapExpression().getValue().substituteVar(arg.first, arg.second);
+        substVarTypes(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index e66809e..2c68697 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -138,50 +139,43 @@
             switch (physOp.getOperatorTag()) {
                 case BROADCAST_EXCHANGE:
                 case ONE_TO_ONE_EXCHANGE:
-                case RANDOM_MERGE_EXCHANGE: {
+                case RANDOM_MERGE_EXCHANGE:
+                case SEQUENTIAL_MERGE_EXCHANGE:
                     // No variables used.
                     break;
-                }
-                case HASH_PARTITION_EXCHANGE: {
-                    HashPartitionExchangePOperator concreteOp = (HashPartitionExchangePOperator) physOp;
-                    usedVariables.addAll(concreteOp.getHashFields());
+                case HASH_PARTITION_EXCHANGE:
+                    HashPartitionExchangePOperator hashPartitionPOp = (HashPartitionExchangePOperator) physOp;
+                    usedVariables.addAll(hashPartitionPOp.getHashFields());
                     break;
-                }
-                case HASH_PARTITION_MERGE_EXCHANGE: {
-                    HashPartitionMergeExchangePOperator concreteOp = (HashPartitionMergeExchangePOperator) physOp;
-                    usedVariables.addAll(concreteOp.getPartitionFields());
-                    for (OrderColumn orderCol : concreteOp.getOrderColumns()) {
+                case HASH_PARTITION_MERGE_EXCHANGE:
+                    HashPartitionMergeExchangePOperator hashMergePOp = (HashPartitionMergeExchangePOperator) physOp;
+                    usedVariables.addAll(hashMergePOp.getPartitionFields());
+                    for (OrderColumn orderCol : hashMergePOp.getOrderColumns()) {
                         usedVariables.add(orderCol.getColumn());
                     }
                     break;
-                }
-                case SORT_MERGE_EXCHANGE: {
-                    SortMergeExchangePOperator concreteOp = (SortMergeExchangePOperator) physOp;
-                    for (OrderColumn orderCol : concreteOp.getSortColumns()) {
+                case SORT_MERGE_EXCHANGE:
+                    SortMergeExchangePOperator sortMergePOp = (SortMergeExchangePOperator) physOp;
+                    for (OrderColumn orderCol : sortMergePOp.getSortColumns()) {
                         usedVariables.add(orderCol.getColumn());
                     }
                     break;
-                }
-                case RANGE_PARTITION_EXCHANGE: {
-                    RangePartitionExchangePOperator concreteOp = (RangePartitionExchangePOperator) physOp;
-                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
+                case RANGE_PARTITION_EXCHANGE:
+                    RangePartitionExchangePOperator rangePartitionPOp = (RangePartitionExchangePOperator) physOp;
+                    for (OrderColumn partCol : rangePartitionPOp.getPartitioningFields()) {
                         usedVariables.add(partCol.getColumn());
                     }
                     break;
-                }
-                case RANGE_PARTITION_MERGE_EXCHANGE: {
-                    RangePartitionMergeExchangePOperator concreteOp = (RangePartitionMergeExchangePOperator) physOp;
-                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
+                case RANGE_PARTITION_MERGE_EXCHANGE:
+                    RangePartitionMergeExchangePOperator rangeMergePOp = (RangePartitionMergeExchangePOperator) physOp;
+                    for (OrderColumn partCol : rangeMergePOp.getPartitioningFields()) {
                         usedVariables.add(partCol.getColumn());
                     }
                     break;
-                }
-                case RANDOM_PARTITION_EXCHANGE: {
+                case RANDOM_PARTITION_EXCHANGE:
                     break;
-                }
-                default: {
+                default:
                     throw new AlgebricksException("Unhandled physical operator tag '" + physOp.getOperatorTag() + "'.");
-                }
             }
         }
         return null;
@@ -439,6 +433,12 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        op.getRangeMapExpression().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
index 78e96a4..0c08369 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -29,16 +30,20 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 
@@ -67,16 +72,26 @@
     }
 
     @Override
-    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
-            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
-        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator sortOp,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext ctx) {
+        if (sortOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
             if (orderProp == null) {
-                computeLocalProperties(op);
+                computeLocalProperties(sortOp);
             }
-            StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(
-                    IPartitioningProperty.UNPARTITIONED, Collections.singletonList(orderProp)) };
-            return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+            StructuralPropertiesVector[] requiredProp = new StructuralPropertiesVector[1];
+            IPartitioningProperty partitioning;
+            INodeDomain targetNodeDomain = ctx.getComputationNodeDomain();
+            if (isFullParallel((AbstractLogicalOperator) sortOp, targetNodeDomain, ctx)) {
+                // partitioning requirement: input data is re-partitioned on sort columns (global ordering)
+                // TODO(ali): static range map implementation should be fixed to require ORDERED_PARTITION and come here
+                partitioning = new OrderedPartitionedProperty(Arrays.asList(sortColumns), targetNodeDomain);
+            } else {
+                // partitioning requirement: input data is unpartitioned (i.e. must be merged at one site)
+                partitioning = IPartitioningProperty.UNPARTITIONED;
+            }
+            // local requirement: each partition must be locally ordered
+            requiredProp[0] = new StructuralPropertiesVector(partitioning, Collections.singletonList(orderProp));
+            return new PhysicalRequirements(requiredProp, IPartitioningRequirementsCoordinator.NO_COORDINATION);
         } else {
             return emptyUnaryRequirements();
         }
@@ -123,4 +138,27 @@
     public boolean expensiveThanMaterialization() {
         return true;
     }
+
+    /**
+     * When true, the sort operator requires ORDERED_PARTITION (only applicable to dynamic version for now).
+     * Conditions:
+     * 1. Execution mode == partitioned
+     * 2. Dynamic range map was not disabled by some checks
+     * 3. User didn't disable it
+     * 4. User didn't provide static range map
+     * 5. Physical sort operator is not in-memory
+     * 6. There are at least two partitions in the cluster
+     * @param sortOp the sort operator
+     * @param clusterDomain the partitions specification of the cluster
+     * @param ctx optimization context
+     * @return true if the sort operator should be full parallel sort, false otherwise.
+     */
+    private boolean isFullParallel(AbstractLogicalOperator sortOp, INodeDomain clusterDomain,
+            IOptimizationContext ctx) {
+        return sortOp.getAnnotations().get(OperatorAnnotations.USE_DYNAMIC_RANGE) != Boolean.FALSE
+                && !sortOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)
+                && sortOp.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.STABLE_SORT
+                && clusterDomain.cardinality() != null && clusterDomain.cardinality() > 1
+                && ctx.getPhysicalOptimizationConfig().getSortParallel();
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java
new file mode 100644
index 0000000..11c584e
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ForwardOperatorDescriptor;
+
+/**
+ * <pre>
+ * {@see {@link ForwardOperator} and {@link ForwardOperatorDescriptor}}
+ * idx0: Input data source --
+ *                           |-- forward op.
+ * idx1: RangeMap generator--
+ * </pre>
+ */
+public class ForwardPOperator extends AbstractPhysicalOperator {
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.FORWARD;
+    }
+
+    /**
+     * Forward operator requires that the global aggregate operator broadcasts the range map. No required properties at
+     * the data source input.
+     * @param op {@see {@link ForwardOperator}}
+     * @param requiredByParent parent's requirements, which are not enforced for now, as we only explore one plan
+     * @param context the optimization context
+     * @return broadcast requirement at input 1; empty requirements at input 0; No coordination between the two.
+     */
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector requiredByParent, IOptimizationContext context) {
+        // broadcast the range map to the cluster node domain
+        INodeDomain targetDomain = context.getComputationNodeDomain();
+        List<ILocalStructuralProperty> noProp = new ArrayList<>();
+        StructuralPropertiesVector[] requiredAtInputs = new StructuralPropertiesVector[2];
+        requiredAtInputs[0] = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+        requiredAtInputs[1] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(targetDomain), noProp);
+        return new PhysicalRequirements(requiredAtInputs, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+    }
+
+    /**
+     * Forward operator delivers whatever properties delivered by the input located at index = 0 (tuples source op).
+     * Subtree at index 0 must compute its delivered properties before any call to this method
+     * @param op forward logical operator
+     * @param context {@link IOptimizationContext}
+     * @throws AlgebricksException
+     */
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator dataSourceOperator = op.getInputs().get(0).getValue();
+        deliveredProperties = dataSourceOperator.getDeliveredPhysicalProperties().clone();
+    }
+
+    /**
+     * The output record descriptor of forward operator is the same as the output record descriptor of the data source
+     * which is located at index 0.
+     * @param builder Hyracks job builder
+     * @param context job generation context
+     * @param op {@see {@link ForwardOperator}}
+     * @param propagatedSchema not used
+     * @param inputSchemas schemas of all inputs
+     * @param outerPlanSchema not used
+     * @throws AlgebricksException
+     */
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        ForwardOperator forwardOp = (ForwardOperator) op;
+        RecordDescriptor dataInputDescriptor = JobGenHelper.mkRecordDescriptor(
+                context.getTypeEnvironment(forwardOp.getInputs().get(0).getValue()), inputSchemas[0], context);
+        ForwardOperatorDescriptor forwardDescriptor =
+                new ForwardOperatorDescriptor(builder.getJobSpec(), forwardOp.getRangeMapKey(), dataInputDescriptor);
+        builder.contributeHyracksOperator(forwardOp, forwardDescriptor);
+        ILogicalOperator dataSource = forwardOp.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(dataSource, 0, forwardOp, 0);
+        ILogicalOperator rangemapSource = forwardOp.getInputs().get(1).getValue();
+        builder.contributeGraphEdge(rangemapSource, 0, forwardOp, 1);
+    }
+
+    @Override
+    public Pair<int[], int[]> getInputOutputDependencyLabels(ILogicalOperator op) {
+        int[] outputDependencyLabels = new int[] { 1 };
+        int[] inputDependencyLabels = new int[] { 1, 0 };
+        return new Pair<>(inputDependencyLabels, outputDependencyLabels);
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public boolean expensiveThanMaterialization() {
+        return false;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
index 6630d32..aeb9ac7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
@@ -42,27 +42,40 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
-import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.DynamicFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 
 public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap;
+    private final boolean rangeMapIsComputedAtRunTime;
+    private final String rangeMapKeyInContext;
 
-    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
-            IRangeMap rangeMap) {
+    private RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, RangeMap rangeMap,
+            boolean rangeMapIsComputedAtRunTime, String rangeMapKeyInContext) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
         this.rangeMap = rangeMap;
+        this.rangeMapIsComputedAtRunTime = rangeMapIsComputedAtRunTime;
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+    }
+
+    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, String rangeMapKeyInContext,
+            INodeDomain domain) {
+        this(partitioningFields, domain, null, true, rangeMapKeyInContext);
+    }
+
+    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+            RangeMap rangeMap) {
+        this(partitioningFields, domain, rangeMap, false, "");
     }
 
     @Override
@@ -80,8 +93,7 @@
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        IPartitioningProperty p =
-                new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain);
+        IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<>(partitioningFields), domain);
         this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
     }
 
@@ -97,32 +109,31 @@
         int n = partitioningFields.size();
         int[] sortFields = new int[n];
         IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
-
-        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
-        INormalizedKeyComputerFactory nkcf = null;
-
         IVariableTypeEnvironment env = context.getTypeEnvironment(op);
         int i = 0;
         for (OrderColumn oc : partitioningFields) {
             LogicalVariable var = oc.getColumn();
             sortFields[i] = opSchema.findVariable(var);
             Object type = env.getVarType(var);
-            OrderKind order = oc.getOrder();
-            if (i == 0 && nkcfProvider != null && type != null) {
-                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderKind.ASC);
-            }
             IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
             comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
-        IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, tpcf);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+        FieldRangePartitionComputerFactory partitionerFactory;
+        if (rangeMapIsComputedAtRunTime) {
+            partitionerFactory = new DynamicFieldRangePartitionComputerFactory(sortFields, comps, rangeMapKeyInContext,
+                    op.getSourceLocation());
+        } else {
+            partitionerFactory = new StaticFieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+        }
+
+        IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, partitionerFactory);
+        return new Pair<>(conn, null);
     }
 
     @Override
     public String toString() {
-        return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+        final String splitCount = rangeMap == null ? "" : " SPLIT COUNT:" + Integer.toString(rangeMap.getSplitCount());
+        return getOperatorTag().toString() + " " + partitioningFields + splitCount;
     }
-
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
index ec32a53..b015193 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
@@ -53,18 +53,18 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 
 public class RangePartitionMergeExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap;
 
     public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
-            IRangeMap rangeMap) {
+            RangeMap rangeMap) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
         this.rangeMap = rangeMap;
@@ -143,7 +143,7 @@
             comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+        ITuplePartitionComputerFactory tpcf = new StaticFieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
         IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java
new file mode 100644
index 0000000..df0b446
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.connectors.MToOneSequentialMergingConnectorDescriptor;
+
+public class SequentialMergeExchangePOperator extends AbstractExchangePOperator {
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.SEQUENTIAL_MERGE_EXCHANGE;
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        List<ILocalStructuralProperty> childLocalProps = childOp.getDeliveredPhysicalProperties().getLocalProperties();
+        List<ILocalStructuralProperty> localProperties;
+        if (childLocalProps != null) {
+            localProperties = new ArrayList<>(childLocalProps);
+        } else {
+            localProperties = new ArrayList<>(0);
+        }
+
+        deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, localProperties);
+    }
+
+    @Override
+    public Pair<IConnectorDescriptor, IHyracksJobBuilder.TargetConstraint> createConnectorDescriptor(
+            IConnectorDescriptorRegistry spec, ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context)
+            throws AlgebricksException {
+        IConnectorDescriptor connector = new MToOneSequentialMergingConnectorDescriptor(spec);
+        return new Pair<>(connector, IHyracksJobBuilder.TargetConstraint.ONE);
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 99ed738..77f052e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -465,6 +466,13 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent)
+                .append("forward: range-map = " + op.getRangeMapExpression().getValue().accept(exprVisitor, indent));
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("sink");
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index f1f1f3b..4a17cc6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -44,6 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -647,6 +648,14 @@
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent).append("\"operator\": \"forward\"");
+        addIndent(indent).append("\"expressions\": \""
+                + op.getRangeMapExpression().getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("\"operator\": \"sink\"");
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
index e7e98a5..b006a1e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
@@ -121,20 +121,7 @@
         Iterator<OrderColumn> currentColumnIterator = orderColumns.iterator();
 
         // Returns true if requiredColumnIterator is a prefix of currentColumnIterator.
-        return isPrefixOf(requiredColumnIterator, currentColumnIterator);
-    }
-
-    private <T> boolean isPrefixOf(Iterator<T> requiredColumnIterator, Iterator<T> currentColumnIterator) {
-        while (requiredColumnIterator.hasNext()) {
-            T oc = requiredColumnIterator.next();
-            if (!currentColumnIterator.hasNext()) {
-                return false;
-            }
-            if (!oc.equals(currentColumnIterator.next())) {
-                return false;
-            }
-        }
-        return true;
+        return PropertiesUtil.isPrefixOf(requiredColumnIterator, currentColumnIterator);
     }
 
     // Gets normalized  ordering columns, where each column variable is a representative variable of its equivalence
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
index f2fed13..1c00e45 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
@@ -200,7 +200,7 @@
      * @param target
      * @return true iff pref is a prefix of target
      */
-    private static <T> boolean isPrefixOf(Iterator<T> pref, Iterator<T> target) {
+    public static <T> boolean isPrefixOf(Iterator<T> pref, Iterator<T> target) {
         while (pref.hasNext()) {
             T v = pref.next();
             if (!target.hasNext()) {
@@ -213,50 +213,65 @@
         return true;
     }
 
+    /**
+     * Normalizes or reduces the order columns argument based on the functional dependencies argument. The caller is
+     * responsible for taking caution as to how to handle the returned object since this method either returns the same
+     * object that is passed or returns a new object.
+     * @param orderColumns the order columns that are to be normalized
+     * @param functionalDependencies {@link FunctionalDependency}
+     * @return a new normalized object if normalization is applied. Otherwise, the same argument object is returned.
+     */
     public static List<OrderColumn> applyFDsToOrderColumns(List<OrderColumn> orderColumns,
-            List<FunctionalDependency> fds) {
-        // the set of vars. is ordered
-        // so we try the variables in order from last to first
-        if (fds == null || fds.isEmpty()) {
+            List<FunctionalDependency> functionalDependencies) {
+        if (functionalDependencies == null || functionalDependencies.isEmpty()) {
             return orderColumns;
         }
 
+        // the set of vars. is ordered
+        // so we try the variables in order from last to first
         int deleted = 0;
+        boolean[] removedColumns = new boolean[orderColumns.size()];
         for (int i = orderColumns.size() - 1; i >= 0; i--) {
-            for (FunctionalDependency fdep : fds) {
-                if (impliedByPrefix(orderColumns, i, fdep)) {
-                    orderColumns.set(i, null);
+            for (FunctionalDependency functionalDependency : functionalDependencies) {
+                if (impliedByPrefix(orderColumns, i, functionalDependency)) {
+                    removedColumns[i] = true;
                     deleted++;
                     break;
                 }
             }
         }
-        List<OrderColumn> norm = new ArrayList<>(orderColumns.size() - deleted);
-        for (OrderColumn oc : orderColumns) {
-            if (oc != null) {
-                norm.add(oc);
+        List<OrderColumn> normalizedColumns = new ArrayList<>(orderColumns.size() - deleted);
+        for (int i = 0; i < orderColumns.size(); i++) {
+            if (!removedColumns[i]) {
+                normalizedColumns.add(orderColumns.get(i));
             }
         }
-        return norm;
+
+        return normalizedColumns;
     }
 
+    /**
+     * Normalizes or reduces the order columns argument based on the equivalenceClasses argument. The caller is
+     * responsible for taking caution as to how to handle the returned object since this method either returns the same
+     * object that is passed or returns a new object.
+     * @param orderColumns the order columns that are to be normalized
+     * @param equivalenceClasses {@link EquivalenceClass}
+     * @return a new normalized object if normalization is applied. Otherwise, the same argument object is returned.
+     */
     public static List<OrderColumn> replaceOrderColumnsByEqClasses(List<OrderColumn> orderColumns,
             Map<LogicalVariable, EquivalenceClass> equivalenceClasses) {
         if (equivalenceClasses == null || equivalenceClasses.isEmpty()) {
             return orderColumns;
         }
         List<OrderColumn> norm = new ArrayList<>();
-        for (OrderColumn v : orderColumns) {
-            EquivalenceClass ec = equivalenceClasses.get(v.getColumn());
-            if (ec == null) {
-                norm.add(v);
-            } else {
-                if (ec.representativeIsConst()) {
-                    // trivially satisfied, so the var. can be removed
-                } else {
-                    norm.add(new OrderColumn(ec.getVariableRepresentative(), v.getOrder()));
-                }
+        for (OrderColumn orderColumn : orderColumns) {
+            EquivalenceClass columnEQClass = equivalenceClasses.get(orderColumn.getColumn());
+            if (columnEQClass == null) {
+                norm.add(orderColumn);
+            } else if (!columnEQClass.representativeIsConst()) {
+                norm.add(new OrderColumn(columnEQClass.getVariableRepresentative(), orderColumn.getOrder()));
             }
+            // else columnEQClass rep. is constant, i.e. trivially satisfied, so the var. can be removed
         }
         return norm;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index deb98b0..548a29f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -124,4 +125,5 @@
 
     public R visitTokenizeOperator(TokenizeOperator op, T arg) throws AlgebricksException;
 
+    public R visitForwardOperator(ForwardOperator op, T arg) throws AlgebricksException;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 8779777..15bb54b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -25,4 +25,6 @@
 
     public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
+    public static final int SORT_SAMPLES = 100;
+    public static final boolean SORT_PARALLEL = true;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
index 6fa378b..e3b1868 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
 import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToOneSequentialMergingConnectorDescriptor;
 
 public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
     private static final long serialVersionUID = 1L;
@@ -33,7 +34,8 @@
     @Override
     public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
             int[] fanouts) {
-        if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
+        if (c instanceof MToNPartitioningMergingConnectorDescriptor
+                || c instanceof MToOneSequentialMergingConnectorDescriptor) {
             return senderSideMaterializePolicy;
         } else {
             return pipeliningPolicy;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index 8eb9b90..a2a0ca1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -37,29 +37,20 @@
 
 public class HeuristicOptimizer {
 
-    public static PhysicalOperatorTag[] hyracksOperators =
-            new PhysicalOperatorTag[] { PhysicalOperatorTag.DATASOURCE_SCAN, PhysicalOperatorTag.BTREE_SEARCH,
-                    PhysicalOperatorTag.EXTERNAL_GROUP_BY, PhysicalOperatorTag.HASH_GROUP_BY,
-                    PhysicalOperatorTag.HDFS_READER, PhysicalOperatorTag.HYBRID_HASH_JOIN,
-                    PhysicalOperatorTag.IN_MEMORY_HASH_JOIN, PhysicalOperatorTag.NESTED_LOOP,
-                    PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY, PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY,
-                    PhysicalOperatorTag.REPLICATE, PhysicalOperatorTag.STABLE_SORT, PhysicalOperatorTag.UNION_ALL };
-    public static PhysicalOperatorTag[] hyraxOperatorsBelowWhichJobGenIsDisabled = new PhysicalOperatorTag[] {};
-
-    public static boolean isHyracksOp(PhysicalOperatorTag opTag) {
-        for (PhysicalOperatorTag t : hyracksOperators) {
-            if (t == opTag) {
-                return true;
-            }
-        }
-        return false;
-    }
-
     private final IOptimizationContext context;
     private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
     private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
     private final ILogicalPlan plan;
 
+    private static final PhysicalOperatorTag[] hyracksOperators = new PhysicalOperatorTag[] {
+            PhysicalOperatorTag.DATASOURCE_SCAN, PhysicalOperatorTag.BTREE_SEARCH,
+            PhysicalOperatorTag.EXTERNAL_GROUP_BY, PhysicalOperatorTag.HASH_GROUP_BY, PhysicalOperatorTag.HDFS_READER,
+            PhysicalOperatorTag.HYBRID_HASH_JOIN, PhysicalOperatorTag.IN_MEMORY_HASH_JOIN,
+            PhysicalOperatorTag.NESTED_LOOP, PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY,
+            PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY, PhysicalOperatorTag.REPLICATE, PhysicalOperatorTag.STABLE_SORT,
+            PhysicalOperatorTag.UNION_ALL, PhysicalOperatorTag.FORWARD };
+    public static final PhysicalOperatorTag[] hyraxOperatorsBelowWhichJobGenIsDisabled = new PhysicalOperatorTag[] {};
+
     public HeuristicOptimizer(ILogicalPlan plan,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
@@ -70,6 +61,15 @@
         this.physicalRewrites = physicalRewrites;
     }
 
+    public static boolean isHyracksOp(PhysicalOperatorTag opTag) {
+        for (PhysicalOperatorTag t : hyracksOperators) {
+            if (t == opTag) {
+                return true;
+            }
+        }
+        return false;
+    }
+
     public void optimize() throws AlgebricksException {
         if (plan == null) {
             return;
@@ -129,7 +129,6 @@
         if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting physical optimizations.\n");
         }
-        // PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(plan);
         runOptimizationSets(plan, physicalRewrites);
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 07ff0ab..d879d36 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -20,6 +20,8 @@
 
 import java.util.Properties;
 
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
+
 public class PhysicalOptimizationConfig {
     private static final int MB = 1048576;
 
@@ -31,10 +33,11 @@
     private static final String MAX_FRAMES_FOR_TEXTSEARCH = "MAX_FRAMES_FOR_TEXTSEARCH";
     private static final String FUDGE_FACTOR = "FUDGE_FACTOR";
     private static final String MAX_RECORDS_PER_FRAME = "MAX_RECORDS_PER_FRAME";
-
     private static final String DEFAULT_HASH_GROUP_TABLE_SIZE = "DEFAULT_HASH_GROUP_TABLE_SIZE";
     private static final String DEFAULT_EXTERNAL_GROUP_TABLE_SIZE = "DEFAULT_EXTERNAL_GROUP_TABLE_SIZE";
     private static final String DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE = "DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE";
+    private static final String SORT_PARALLEL = "SORT_PARALLEL";
+    private static final String SORT_SAMPLES = "SORT_SAMPLES";
 
     private Properties properties = new Properties();
 
@@ -143,6 +146,22 @@
         setInt(DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE, tableSize);
     }
 
+    public boolean getSortParallel() {
+        return getBoolean(SORT_PARALLEL, AlgebricksConfig.SORT_PARALLEL);
+    }
+
+    public void setSortParallel(boolean sortParallel) {
+        setBoolean(SORT_PARALLEL, sortParallel);
+    }
+
+    public int getSortSamples() {
+        return getInt(SORT_SAMPLES, AlgebricksConfig.SORT_SAMPLES);
+    }
+
+    public void setSortSamples(int sortSamples) {
+        setInt(SORT_SAMPLES, sortSamples);
+    }
+
     private void setInt(String property, int value) {
         properties.setProperty(property, Integer.toString(value));
     }
@@ -167,4 +186,16 @@
             return Double.parseDouble(value);
     }
 
+    private void setBoolean(String property, boolean value) {
+        properties.setProperty(property, Boolean.toString(value));
+    }
+
+    private boolean getBoolean(String property, boolean defaultValue) {
+        String value = properties.getProperty(property);
+        if (value == null) {
+            return defaultValue;
+        } else {
+            return Boolean.parseBoolean(value);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
index 8ada0ac..d6895e3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
@@ -29,12 +29,16 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.api.constraints.Constraint;
+import org.apache.hyracks.api.constraints.expressions.ConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionLocationExpression;
 import org.apache.hyracks.api.dataflow.ActivityId;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.IActivity;
@@ -45,15 +49,13 @@
 
 public class DotFormatGenerator {
 
-    private DotFormatGenerator() {
-    }
+    private final LogicalOperatorDotVisitor dotVisitor = new LogicalOperatorDotVisitor();
 
     /**
-     * Generates DOT format for {@link JobActivityGraph} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link JobActivityGraph} that can be visualized using any DOT format visualizer.
      *
      * @param jobActivityGraph The job activity graph
-     * @return DOT format
+     * @return DOT format plan
      */
     public static String generate(final JobActivityGraph jobActivityGraph) {
         final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("JobActivityGraph"));
@@ -146,92 +148,74 @@
     }
 
     /**
-     * Generates DOT format for {@link JobSpecification} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link JobSpecification} that can be visualized using any DOT format visualizer.
      *
      * @param jobSpecification The job specification
-     * @return DOT format
+     * @return DOT format plan
      */
     public static String generate(final JobSpecification jobSpecification) {
         final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("JobSpecification"));
         final Map<ConnectorDescriptorId, IConnectorDescriptor> connectorMap = jobSpecification.getConnectorMap();
-        final Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> cOp =
+        final Set<Constraint> constraints = jobSpecification.getUserConstraints();
+        Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> cOp =
                 jobSpecification.getConnectorOperatorMap();
-        ConnectorDescriptorId connectorId;
-        IConnectorDescriptor connector;
-        IOperatorDescriptor leftOperator;
-        IOperatorDescriptor rightOperator;
-        DotFormatBuilder.Node sourceNode;
-        DotFormatBuilder.Node destinationNode;
-        String source;
-        String destination;
-        String edgeLabel;
-        for (Map.Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> entry : cOp
-                .entrySet()) {
-            connectorId = entry.getKey();
-            connector = connectorMap.get(connectorId);
-            edgeLabel = connector.getClass().getName().substring(connector.getClass().getName().lastIndexOf(".") + 1);
-            edgeLabel += "-" + connectorId;
-            leftOperator = entry.getValue().getLeft().getLeft();
-            rightOperator = entry.getValue().getRight().getLeft();
-            source = leftOperator.getClass().getName()
-                    .substring(leftOperator.getClass().getName().lastIndexOf(".") + 1);
-            sourceNode =
-                    graphBuilder.createNode(DotFormatBuilder.StringValue.of(leftOperator.getOperatorId().toString()),
-                            DotFormatBuilder.StringValue.of(leftOperator.toString() + "-" + source));
-            destination = rightOperator.getClass().getName()
-                    .substring(rightOperator.getClass().getName().lastIndexOf(".") + 1);
-            destinationNode =
-                    graphBuilder.createNode(DotFormatBuilder.StringValue.of(rightOperator.getOperatorId().toString()),
-                            DotFormatBuilder.StringValue.of(rightOperator.toString() + "-" + destination));
-            graphBuilder.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of(edgeLabel));
-        }
-
+        cOp.forEach((connId, srcAndDest) -> addToGraph(graphBuilder, constraints, connectorMap, connId, srcAndDest));
         return graphBuilder.getDotDocument();
     }
 
     /**
-     * Generates DOT format for {@link ILogicalPlan} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link ILogicalPlan} that can be visualized using any DOT format visualizer.
      *
      * @param plan  The logical plan
-     * @param dotVisitor    The DOT visitor
-     * @return DOT format
-     * @throws AlgebricksException
+     * @param showDetails whether to show the details of the operator like physical properties
+     * @return DOT format plan
+     * @throws AlgebricksException When one operator throws an exception while visiting it.
      */
-    public static String generate(ILogicalPlan plan, LogicalOperatorDotVisitor dotVisitor) throws AlgebricksException {
-        final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("Plan"));
+    public String generate(ILogicalPlan plan, boolean showDetails) throws AlgebricksException {
         ILogicalOperator root = plan.getRoots().get(0).getValue();
-        generateNode(graphBuilder, root, dotVisitor, new HashSet<>());
+        return generate(root, showDetails);
+    }
+
+    /**
+     * Generates DOT format plan considering "startingOp" as the root operator.
+     *
+     * @param startingOp the starting operator
+     * @param showDetails whether to show the details of the operator like physical properties
+     * @return DOT format plan
+     * @throws AlgebricksException When one operator throws an exception while visiting it.
+     */
+    public String generate(ILogicalOperator startingOp, boolean showDetails) throws AlgebricksException {
+        final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("Plan"));
+        generateNode(graphBuilder, startingOp, showDetails, new HashSet<>());
         return graphBuilder.getDotDocument();
     }
 
-    public static void generateNode(DotFormatBuilder dotBuilder, ILogicalOperator op,
-            LogicalOperatorDotVisitor dotVisitor, Set<ILogicalOperator> operatorsVisited) throws AlgebricksException {
-        DotFormatBuilder.StringValue destinationNodeLabel = formatStringOf(op, dotVisitor);
+    private void generateNode(DotFormatBuilder dotBuilder, ILogicalOperator op, boolean showDetails,
+            Set<ILogicalOperator> operatorsVisited) throws AlgebricksException {
+        DotFormatBuilder.StringValue destinationNodeLabel = formatStringOf(op, showDetails);
         DotFormatBuilder.Node destinationNode = dotBuilder
                 .createNode(DotFormatBuilder.StringValue.of(Integer.toString(op.hashCode())), destinationNodeLabel);
         DotFormatBuilder.StringValue sourceNodeLabel;
         DotFormatBuilder.Node sourceNode;
         for (Mutable<ILogicalOperator> child : op.getInputs()) {
-            sourceNodeLabel = formatStringOf(child.getValue(), dotVisitor);
+            sourceNodeLabel = formatStringOf(child.getValue(), showDetails);
             sourceNode = dotBuilder.createNode(
                     DotFormatBuilder.StringValue.of(Integer.toString(child.getValue().hashCode())), sourceNodeLabel);
             dotBuilder.createEdge(sourceNode, destinationNode);
             if (!operatorsVisited.contains(child.getValue())) {
-                generateNode(dotBuilder, child.getValue(), dotVisitor, operatorsVisited);
+                generateNode(dotBuilder, child.getValue(), showDetails, operatorsVisited);
             }
         }
         if (((AbstractLogicalOperator) op).hasNestedPlans()) {
             ILogicalOperator nestedOperator;
             for (ILogicalPlan nestedPlan : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
                 nestedOperator = nestedPlan.getRoots().get(0).getValue();
-                sourceNodeLabel = formatStringOf(nestedOperator, dotVisitor);
+                sourceNodeLabel = formatStringOf(nestedOperator, showDetails);
                 sourceNode = dotBuilder.createNode(
                         DotFormatBuilder.StringValue.of(Integer.toString(nestedOperator.hashCode())), sourceNodeLabel);
                 dotBuilder.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of("subplan"));
                 if (!operatorsVisited.contains(nestedOperator)) {
-                    generateNode(dotBuilder, nestedOperator, dotVisitor, operatorsVisited);
+                    generateNode(dotBuilder, nestedOperator, showDetails, operatorsVisited);
                 }
             }
         }
@@ -246,7 +230,7 @@
             sourceNode = destinationNode;
             for (int i = 0; i < replicateOperator.getOutputs().size(); i++) {
                 replicateOutput = replicateOperator.getOutputs().get(i).getValue();
-                destinationNodeLabel = formatStringOf(replicateOutput, dotVisitor);
+                destinationNodeLabel = formatStringOf(replicateOutput, showDetails);
                 destinationNode = dotBuilder.createNode(
                         DotFormatBuilder.StringValue.of(Integer.toString(replicateOutput.hashCode())),
                         destinationNodeLabel);
@@ -261,16 +245,52 @@
         operatorsVisited.add(op);
     }
 
-    private static DotFormatBuilder.StringValue formatStringOf(ILogicalOperator operator,
-            LogicalOperatorDotVisitor dotVisitor) throws AlgebricksException {
-        String formattedString = operator.accept(dotVisitor, null).trim();
-        IPhysicalOperator physicalOperator = ((AbstractLogicalOperator) operator).getPhysicalOperator();
-        if (physicalOperator != null) {
-            formattedString += "\\n" + physicalOperator.toString().trim() + " |" + operator.getExecutionMode() + "|";
-        } else {
-            formattedString += "\\n|" + operator.getExecutionMode() + "|";
-        }
-
+    private DotFormatBuilder.StringValue formatStringOf(ILogicalOperator operator, boolean showDetails)
+            throws AlgebricksException {
+        String formattedString = operator.accept(dotVisitor, showDetails).trim();
         return DotFormatBuilder.StringValue.of(formattedString);
     }
+
+    private static void addToGraph(DotFormatBuilder graph, Set<Constraint> constraints,
+            Map<ConnectorDescriptorId, IConnectorDescriptor> connMap, ConnectorDescriptorId connId,
+            Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> srcAndDest) {
+        IConnectorDescriptor connector = connMap.get(connId);
+        String edgeLabel;
+        edgeLabel = connector.getClass().getName().substring(connector.getClass().getName().lastIndexOf(".") + 1);
+        edgeLabel += "-" + connId;
+        IOperatorDescriptor sourceOp = srcAndDest.getLeft().getLeft();
+        IOperatorDescriptor destOp = srcAndDest.getRight().getLeft();
+        StringBuilder source = new StringBuilder(
+                sourceOp.getClass().getName().substring(sourceOp.getClass().getName().lastIndexOf(".") + 1));
+        StringBuilder destination = new StringBuilder(
+                destOp.getClass().getName().substring(destOp.getClass().getName().lastIndexOf(".") + 1));
+        // constraints
+        for (Constraint constraint : constraints) {
+            LValueConstraintExpression lvalue = constraint.getLValue();
+            if (lvalue.getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
+                PartitionCountExpression count = (PartitionCountExpression) lvalue;
+                if (count.getOperatorDescriptorId().equals(sourceOp.getOperatorId())) {
+                    source.append("\n").append(constraint);
+                }
+                if (count.getOperatorDescriptorId().equals(destOp.getOperatorId())) {
+                    destination.append("\n").append(constraint);
+                }
+            } else if (lvalue.getTag() == ConstraintExpression.ExpressionTag.PARTITION_LOCATION) {
+                PartitionLocationExpression location = (PartitionLocationExpression) lvalue;
+                if (location.getOperatorDescriptorId().equals(sourceOp.getOperatorId())) {
+                    source.append("\n").append(constraint);
+                }
+                if (location.getOperatorDescriptorId().equals(destOp.getOperatorId())) {
+                    destination.append("\n").append(constraint);
+                }
+            }
+        }
+        DotFormatBuilder.Node sourceNode =
+                graph.createNode(DotFormatBuilder.StringValue.of(sourceOp.getOperatorId().toString()),
+                        DotFormatBuilder.StringValue.of(sourceOp.toString() + "-" + source));
+        DotFormatBuilder.Node destinationNode =
+                graph.createNode(DotFormatBuilder.StringValue.of(destOp.getOperatorId().toString()),
+                        DotFormatBuilder.StringValue.of(destOp.toString() + "-" + destination));
+        graph.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of(edgeLabel));
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 2cb2d35..113d205 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -18,14 +18,20 @@
  */
 package org.apache.hyracks.algebricks.core.utils;
 
+import static org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType.LOCAL_GROUPING_PROPERTY;
+import static org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType.LOCAL_ORDER_PROPERTY;
+
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -35,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -62,9 +69,14 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String, Void> {
+public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String, Boolean> {
 
     private final StringBuilder stringBuilder;
 
@@ -82,161 +94,214 @@
     }
 
     @Override
-    public String visitAggregateOperator(AggregateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitAggregateOperator(AggregateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("aggregate ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitRunningAggregateOperator(RunningAggregateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitRunningAggregateOperator(RunningAggregateOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("running-aggregate ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void noArgs) throws AlgebricksException {
+    public String visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("empty-tuple-source");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitGroupByOperator(GroupByOperator op, Void noArgs) throws AlgebricksException {
+    public String visitGroupByOperator(GroupByOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("group by").append(op.isGroupAll() ? " (all)" : "").append(" (");
-        pprintVeList(op.getGroupByList());
+        printVariableAndExprList(op.getGroupByList());
         stringBuilder.append(") decor (");
-        pprintVeList(op.getDecorList());
+        printVariableAndExprList(op.getDecorList());
         stringBuilder.append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDistinctOperator(DistinctOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDistinctOperator(DistinctOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("distinct (");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
         stringBuilder.append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitInnerJoinOperator(InnerJoinOperator op, Void noArgs) throws AlgebricksException {
+    public String visitInnerJoinOperator(InnerJoinOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("join (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("left outer join (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void noArgs) throws AlgebricksException {
+    public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("nested tuple source");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitOrderOperator(OrderOperator op, Void noArgs) throws AlgebricksException {
+    public String visitOrderOperator(OrderOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("order ");
         for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
             if (op.getTopK() != -1) {
                 stringBuilder.append("(topK: ").append(op.getTopK()).append(") ");
             }
-            String fst = getOrderString(p.first);
-            stringBuilder.append("(").append(fst).append(", ").append(p.second.getValue().toString()).append(") ");
+            stringBuilder.append("(");
+            switch (p.first.getKind()) {
+                case ASC:
+                    stringBuilder.append("ASC");
+                    break;
+                case DESC:
+                    stringBuilder.append("DESC");
+                    break;
+                default:
+                    final Mutable<ILogicalExpression> expressionRef = p.first.getExpressionRef();
+                    stringBuilder.append(expressionRef == null ? "null" : expressionRef.toString());
+            }
+            stringBuilder.append(", ").append(p.second.getValue().toString()).append(") ");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private String getOrderString(OrderOperator.IOrder first) {
-        switch (first.getKind()) {
-            case ASC:
-                return "ASC";
-            case DESC:
-                return "DESC";
-            default:
-                return first.getExpressionRef().toString();
-        }
-    }
-
     @Override
-    public String visitAssignOperator(AssignOperator op, Void noArgs) throws AlgebricksException {
+    public String visitAssignOperator(AssignOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("assign ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitWriteOperator(WriteOperator op, Void noArgs) throws AlgebricksException {
+    public String visitWriteOperator(WriteOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("write ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDistributeResultOperator(DistributeResultOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDistributeResultOperator(DistributeResultOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("distribute result ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitWriteResultOperator(WriteResultOperator op, Void noArgs) throws AlgebricksException {
+    public String visitWriteResultOperator(WriteResultOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("load ").append(str(op.getDataSource())).append(" from ")
                 .append(op.getPayloadExpression().getValue().toString()).append(" partitioned by ");
-        pprintExprList(op.getKeyExpressions());
+        printExprList(op.getKeyExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSelectOperator(SelectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSelectOperator(SelectOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("select (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitProjectOperator(ProjectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitProjectOperator(ProjectOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("project ").append("(").append(op.getVariables()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSubplanOperator(SubplanOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSubplanOperator(SubplanOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("subplan {}");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnionOperator(UnionAllOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnionOperator(UnionAllOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("union");
         for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> v : op.getVariableMappings()) {
             stringBuilder.append(" (").append(v.first).append(", ").append(v.second).append(", ").append(v.third)
                     .append(")");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitIntersectOperator(IntersectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitIntersectOperator(IntersectOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("intersect (");
         stringBuilder.append('[');
@@ -261,154 +326,183 @@
             stringBuilder.append(']');
         }
         stringBuilder.append("])");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnnestOperator(UnnestOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnnestOperator(UnnestOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("unnest ").append(op.getVariable());
         if (op.getPositionalVariable() != null) {
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("outer-unnest ").append(op.getVariable());
         if (op.getPositionalVariable() != null) {
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnnestMapOperator(UnnestMapOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnnestMapOperator(UnnestMapOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
-        printAbstractUnnestMapOperator(op, "unnest-map");
-        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
-        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        printAbstractUnnestMapOperator(op, "unnest-map", showDetails);
+        appendSelectConditionInformation(op.getSelectCondition());
+        appendLimitInformation(op.getOutputLimit());
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void noArgs)
+    public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Boolean showDetails)
             throws AlgebricksException {
         stringBuilder.setLength(0);
-        printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, "left-outer-unnest-map", showDetails);
         return stringBuilder.toString();
     }
 
-    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
+    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature, boolean show) {
         stringBuilder.append(opSignature).append(" ").append(op.getVariables()).append(" <- ")
                 .append(op.getExpressionRef().getValue().toString());
-        appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSchema(op, show);
+        appendAnnotations(op, show);
+        appendPhysicalOperatorInfo(op, show);
     }
 
     @Override
-    public String visitDataScanOperator(DataSourceScanOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDataScanOperator(DataSourceScanOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("data-scan ").append(op.getProjectVariables()).append("<-").append(op.getVariables())
                 .append(" <- ").append(op.getDataSource());
-        appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
-        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
-        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        appendFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(op.getSelectCondition());
+        appendLimitInformation(op.getOutputLimit());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private void appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
-            List<LogicalVariable> maxFilterVars) {
+    private void appendFilterInformation(List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars) {
         if (minFilterVars != null || maxFilterVars != null) {
-            plan.append(" with filter on");
+            stringBuilder.append(" with filter on");
         }
         if (minFilterVars != null) {
-            plan.append(" min:").append(minFilterVars);
+            stringBuilder.append(" min:").append(minFilterVars);
         }
         if (maxFilterVars != null) {
-            plan.append(" max:").append(maxFilterVars);
+            stringBuilder.append(" max:").append(maxFilterVars);
         }
     }
 
-    private Void appendSelectConditionInformation(StringBuilder plan, Mutable<ILogicalExpression> condition)
-            throws AlgebricksException {
+    private void appendSelectConditionInformation(Mutable<ILogicalExpression> condition) throws AlgebricksException {
         if (condition != null) {
-            plan.append(" condition:").append(condition.getValue().toString());
+            stringBuilder.append(" condition:").append(condition.getValue().toString());
         }
-        return null;
     }
 
-    private Void appendLimitInformation(StringBuilder plan, long outputLimit) throws AlgebricksException {
+    private void appendLimitInformation(long outputLimit) throws AlgebricksException {
         if (outputLimit >= 0) {
-            plan.append(" limit:").append(String.valueOf(outputLimit));
+            stringBuilder.append(" limit:").append(String.valueOf(outputLimit));
         }
-        return null;
     }
 
     @Override
-    public String visitLimitOperator(LimitOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLimitOperator(LimitOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("limit ").append(op.getMaxObjects().getValue().toString());
         ILogicalExpression offset = op.getOffset().getValue();
         if (offset != null) {
             stringBuilder.append(", ").append(offset.toString());
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitExchangeOperator(ExchangeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitExchangeOperator(ExchangeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("exchange");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitScriptOperator(ScriptOperator op, Void noArgs) throws AlgebricksException {
+    public String visitScriptOperator(ScriptOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("script (in: ").append(op.getInputVariables()).append(") (out: ")
                 .append(op.getOutputVariables()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitReplicateOperator(ReplicateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitReplicateOperator(ReplicateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("replicate");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSplitOperator(SplitOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSplitOperator(SplitOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         Mutable<ILogicalExpression> branchingExpression = op.getBranchingExpression();
         stringBuilder.append("split ").append(branchingExpression.getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitMaterializeOperator(MaterializeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitMaterializeOperator(MaterializeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("materialize");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Void noArgs)
-            throws AlgebricksException {
+    public String visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         String header = getIndexOpString(op.getOperation());
         stringBuilder.append(header).append(str(op.getDataSource())).append(" from record: ")
                 .append(op.getPayloadExpression().getValue().toString());
         if (op.getAdditionalNonFilteringExpressions() != null) {
             stringBuilder.append(", meta: ");
-            pprintExprList(op.getAdditionalNonFilteringExpressions());
+            printExprList(op.getAdditionalNonFilteringExpressions());
         }
         stringBuilder.append(" partitioned by ");
-        pprintExprList(op.getPrimaryKeyExpressions());
+        printExprList(op.getPrimaryKeyExpressions());
         if (op.getOperation() == Kind.UPSERT) {
             stringBuilder.append(" out: ([record-before-upsert:").append(op.getBeforeOpRecordVar());
             if (op.getBeforeOpAdditionalNonFilteringVars() != null) {
@@ -419,27 +513,32 @@
         if (op.isBulkload()) {
             stringBuilder.append(" [bulkload]");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void noArgs)
-            throws AlgebricksException {
+    public String visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         String header = getIndexOpString(op.getOperation());
         stringBuilder.append(header).append(op.getIndexName()).append(" on ")
                 .append(str(op.getDataSourceIndex().getDataSource())).append(" from ");
         if (op.getOperation() == Kind.UPSERT) {
             stringBuilder.append(" replace:");
-            pprintExprList(op.getPrevSecondaryKeyExprs());
+            printExprList(op.getPrevSecondaryKeyExprs());
             stringBuilder.append(" with:");
-            pprintExprList(op.getSecondaryKeyExpressions());
+            printExprList(op.getSecondaryKeyExpressions());
         } else {
-            pprintExprList(op.getSecondaryKeyExpressions());
+            printExprList(op.getSecondaryKeyExpressions());
         }
         if (op.isBulkload()) {
             stringBuilder.append(" [bulkload]");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
@@ -452,60 +551,143 @@
             case UPSERT:
                 return "upsert into ";
         }
-        return null;
+        return "";
     }
 
     @Override
-    public String visitTokenizeOperator(TokenizeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitTokenizeOperator(TokenizeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("tokenize ").append(str(op.getTokenizeVars())).append(" <- ");
-        pprintExprList(op.getSecondaryKeyExpressions());
+        printExprList(op.getSecondaryKeyExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSinkOperator(SinkOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSinkOperator(SinkOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("sink");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDelegateOperator(DelegateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDelegateOperator(DelegateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append(op.toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private void pprintExprList(List<Mutable<ILogicalExpression>> expressions) {
+    @Override
+    public String visitForwardOperator(ForwardOperator op, Boolean showDetails) throws AlgebricksException {
+        stringBuilder.setLength(0);
+        stringBuilder.append("forward(").append(op.getRangeMapExpression().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
+        return stringBuilder.toString();
+    }
+
+    private void printExprList(List<Mutable<ILogicalExpression>> expressions) {
+        stringBuilder.append("[");
+        expressions.forEach(exprRef -> stringBuilder.append(exprRef.getValue().toString()).append(", "));
+        stringBuilder.append("]");
+    }
+
+    private void printVariableAndExprList(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> variableExprList) {
         stringBuilder.append("[");
         boolean first = true;
-        for (Mutable<ILogicalExpression> exprRef : expressions) {
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> variableExpressionPair : variableExprList) {
             if (first) {
                 first = false;
             } else {
-                stringBuilder.append(", ");
+                stringBuilder.append("; ");
             }
-            stringBuilder.append(exprRef.getValue().toString());
+            if (variableExpressionPair.first != null) {
+                stringBuilder.append(variableExpressionPair.first).append(" := ").append(variableExpressionPair.second);
+            } else {
+                stringBuilder.append(variableExpressionPair.second.getValue().toString());
+            }
         }
         stringBuilder.append("]");
     }
 
-    private void pprintVeList(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> vePairList) {
-        stringBuilder.append("[");
-        boolean fst = true;
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> ve : vePairList) {
-            if (fst) {
-                fst = false;
-            } else {
-                stringBuilder.append("; ");
-            }
-            if (ve.first != null) {
-                stringBuilder.append(ve.first).append(" := ").append(ve.second);
-            } else {
-                stringBuilder.append(ve.second.getValue().toString());
+    private void appendSchema(AbstractLogicalOperator op, boolean show) {
+        if (show) {
+            stringBuilder.append("\\nSchema: ");
+            final List<LogicalVariable> schema = op.getSchema();
+            stringBuilder.append(schema == null ? "null" : schema);
+        }
+    }
+
+    private void appendAnnotations(AbstractLogicalOperator op, boolean show) {
+        if (show) {
+            final Map<String, Object> annotations = op.getAnnotations();
+            if (!annotations.isEmpty()) {
+                stringBuilder.append("\\nAnnotations: ").append(annotations);
             }
         }
-        stringBuilder.append("]");
+    }
+
+    private void appendPhysicalOperatorInfo(AbstractLogicalOperator op, boolean show) {
+        IPhysicalOperator physicalOp = op.getPhysicalOperator();
+        stringBuilder.append("\\n").append(physicalOp == null ? "null" : physicalOp.toString().trim());
+        stringBuilder.append(", Exec: ").append(op.getExecutionMode());
+        if (show) {
+            IPhysicalPropertiesVector properties = physicalOp == null ? null : physicalOp.getDeliveredProperties();
+            List<ILocalStructuralProperty> localProp = properties == null ? null : properties.getLocalProperties();
+            IPartitioningProperty partitioningProp = properties == null ? null : properties.getPartitioningProperty();
+            if (localProp != null) {
+                stringBuilder.append("\\nProperties in each partition: [");
+                for (ILocalStructuralProperty property : localProp) {
+                    if (property == null) {
+                        stringBuilder.append("null, ");
+                    } else if (property.getPropertyType() == LOCAL_ORDER_PROPERTY) {
+                        stringBuilder.append("ordered by ");
+                    } else if (property.getPropertyType() == LOCAL_GROUPING_PROPERTY) {
+                        stringBuilder.append("group by ");
+                    }
+                    stringBuilder.append(property).append(", ");
+                }
+                stringBuilder.append("]");
+            }
+
+            if (partitioningProp != null) {
+                stringBuilder.append("\\n").append(partitioningProp.getPartitioningType()).append(":");
+                INodeDomain nodeDomain = partitioningProp.getNodeDomain();
+                stringBuilder.append("\\n ");
+                if (nodeDomain != null && nodeDomain.cardinality() != null) {
+                    stringBuilder.append(nodeDomain.cardinality()).append(" partitions. ");
+                }
+                switch (partitioningProp.getPartitioningType()) {
+                    case BROADCAST:
+                        stringBuilder.append("Data is broadcast to partitions.");
+                        break;
+                    case RANDOM:
+                        stringBuilder.append("Data is randomly partitioned.");
+                        break;
+                    case ORDERED_PARTITIONED:
+                        stringBuilder.append("Data is orderly partitioned via a range.");
+                        break;
+                    case UNORDERED_PARTITIONED:
+                        stringBuilder.append("Data is hash partitioned.");
+                        break;
+                    case UNPARTITIONED:
+                        stringBuilder.append("Data is in one place.");
+                }
+                if (nodeDomain instanceof DefaultNodeGroupDomain) {
+                    DefaultNodeGroupDomain nd = (DefaultNodeGroupDomain) nodeDomain;
+                    stringBuilder.append("\\n").append(nd);
+                }
+            }
+        }
+
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 6f7f86a..cdab2f4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -21,9 +21,11 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -40,28 +42,41 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPreSortedDistinctByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPreclusteredGroupByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ForwardPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergeExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.SequentialMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
@@ -90,13 +105,20 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
 import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
 
     private static final String HASH_MERGE = "hash_merge";
     private static final String TRUE_CONSTANT = "true";
     private PhysicalOptimizationConfig physicalOptimizationConfig;
+    private final FunctionIdentifier rangeMapFunction;
+    private final FunctionIdentifier localSamplingFun;
+
+    public EnforceStructuralPropertiesRule(FunctionIdentifier rangeMapFunction, FunctionIdentifier localSamplingFun) {
+        this.rangeMapFunction = rangeMapFunction;
+        this.localSamplingFun = localSamplingFun;
+    }
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -204,6 +226,7 @@
 
         boolean changed = false;
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
         optimizeUsingConstraintsAndEquivClasses(op);
         PhysicalRequirements pr = op.getRequiredPhysicalPropertiesForChildren(required, context);
         IPhysicalPropertiesVector[] reqdProperties = null;
@@ -214,26 +237,19 @@
 
         // compute properties and figure out the domain
         INodeDomain childrenDomain = null;
-        {
-            int j = 0;
-            for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
-                AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
-                // recursive call
-                if (physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context)) {
-                    changed = true;
-                }
-                child.computeDeliveredPhysicalProperties(context);
-                IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
-                if (childrenDomain == null) {
-                    childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
-                } else {
-                    INodeDomain dom2 = delivered.getPartitioningProperty().getNodeDomain();
-                    if (!childrenDomain.sameAs(dom2)) {
-                        childrenDomain = context.getComputationNodeDomain();
-                    }
-                }
-                j++;
+        int j = 0;
+        for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+            changed |= physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context);
+            child.computeDeliveredPhysicalProperties(context);
+            IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+            INodeDomain childDomain = delivered.getPartitioningProperty().getNodeDomain();
+            if (childrenDomain == null) {
+                childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
+            } else if (!childrenDomain.sameAs(childDomain)) {
+                childrenDomain = context.getComputationNodeDomain();
             }
+            j++;
         }
 
         if (reqdProperties != null) {
@@ -252,7 +268,7 @@
         int startChildIndex = getStartChildIndex(op, pr, nestedPlan, context);
         IPartitioningProperty firstDeliveredPartitioning = null;
         // Enforce data properties in a top-down manner.
-        for (int j = 0; j < op.getInputs().size(); j++) {
+        for (j = 0; j < op.getInputs().size(); j++) {
             // Starts from a partitioning-compatible child if any to loop over all children.
             int childIndex = (j + startChildIndex) % op.getInputs().size();
             IPhysicalPropertiesVector requiredProperty = reqdProperties[childIndex];
@@ -555,6 +571,17 @@
         return new MutableObject<ILogicalOperator>(oo);
     }
 
+    /**
+     * Adds exchange operators (connectors) between {@code op} & its child at index {@code childIdx}.
+     * @param op the parent operator that is requiring a specific kind of connector at its child
+     * @param i the child index where we want to have the connector
+     * @param pp the required partitioning property at that child (i.e. the required connector)
+     * @param required the physical properties required at that child (partitioning + local properties)
+     * @param deliveredByChild the physical properties delivered by that child (partitioning + local properties)
+     * @param domain the destination domain of nodes that we want the connector to connect to
+     * @param context {@link IOptimizationContext}
+     * @throws AlgebricksException
+     */
     private void addPartitioningEnforcers(ILogicalOperator op, int i, IPartitioningProperty pp,
             IPhysicalPropertiesVector required, IPhysicalPropertiesVector deliveredByChild, INodeDomain domain,
             IOptimizationContext context) throws AlgebricksException {
@@ -562,52 +589,15 @@
             IPhysicalOperator pop;
             switch (pp.getPartitioningType()) {
                 case UNPARTITIONED: {
-                    List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
-                    if (ordCols.isEmpty()) {
-                        pop = new RandomMergeExchangePOperator();
-                    } else {
-                        if (op.getAnnotations().containsKey(OperatorAnnotations.USE_RANGE_CONNECTOR)) {
-                            IRangeMap rangeMap =
-                                    (IRangeMap) op.getAnnotations().get(OperatorAnnotations.USE_RANGE_CONNECTOR);
-                            pop = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap);
-                        } else {
-                            OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
-                            sortColumns = ordCols.toArray(sortColumns);
-                            pop = new SortMergeExchangePOperator(sortColumns);
-                        }
-                    }
+                    pop = createMergingConnector(op, domain, deliveredByChild);
                     break;
                 }
                 case UNORDERED_PARTITIONED: {
-                    List<LogicalVariable> varList = new ArrayList<>(((UnorderedPartitionedProperty) pp).getColumnSet());
-                    String hashMergeHint = (String) context.getMetadataProvider().getConfig().get(HASH_MERGE);
-                    if (hashMergeHint == null || !hashMergeHint.equalsIgnoreCase(TRUE_CONSTANT)) {
-                        pop = new HashPartitionExchangePOperator(varList, domain);
-                        break;
-                    }
-                    List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
-                    List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
-                    boolean propWasSet = false;
-                    pop = null;
-                    if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
-                        AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
-                        Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
-                        List<FunctionalDependency> fds = context.getFDList(c);
-                        if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
-                            List<OrderColumn> orderColumns =
-                                    getOrderColumnsFromGroupingProperties(reqdLocals, cldLocals);
-                            pop = new HashPartitionMergeExchangePOperator(orderColumns, varList, domain);
-                            propWasSet = true;
-                        }
-                    }
-                    if (!propWasSet) {
-                        pop = new HashPartitionExchangePOperator(varList, domain);
-                    }
+                    pop = createHashConnector(context, deliveredByChild, domain, required, pp, i, op);
                     break;
                 }
                 case ORDERED_PARTITIONED: {
-                    pop = new RangePartitionExchangePOperator(((OrderedPartitionedProperty) pp).getOrderColumns(),
-                            domain, null);
+                    pop = createRangePartitionerConnector((AbstractLogicalOperator) op, domain, pp, i, context);
                     break;
                 }
                 case BROADCAST: {
@@ -640,6 +630,264 @@
         }
     }
 
+    private IPhysicalOperator createMergingConnector(ILogicalOperator parentOp, INodeDomain domain,
+            IPhysicalPropertiesVector deliveredByChild) {
+        IPhysicalOperator mergingConnector;
+        List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
+        if (ordCols.isEmpty()) {
+            IPartitioningProperty partitioningDeliveredByChild = deliveredByChild.getPartitioningProperty();
+            if (partitioningDeliveredByChild.getPartitioningType() == PartitioningType.ORDERED_PARTITIONED) {
+                mergingConnector = new SequentialMergeExchangePOperator();
+            } else {
+                mergingConnector = new RandomMergeExchangePOperator();
+            }
+        } else {
+            if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+                RangeMap rangeMap = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+                mergingConnector = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap);
+            } else {
+                OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
+                sortColumns = ordCols.toArray(sortColumns);
+                mergingConnector = new SortMergeExchangePOperator(sortColumns);
+            }
+        }
+        return mergingConnector;
+    }
+
+    private IPhysicalOperator createHashConnector(IOptimizationContext ctx, IPhysicalPropertiesVector deliveredByChild,
+            INodeDomain domain, IPhysicalPropertiesVector requiredAtChild, IPartitioningProperty rqdPartitioning,
+            int childIndex, ILogicalOperator parentOp) {
+        IPhysicalOperator hashConnector;
+        List<LogicalVariable> vars = new ArrayList<>(((UnorderedPartitionedProperty) rqdPartitioning).getColumnSet());
+        String hashMergeHint = (String) ctx.getMetadataProvider().getConfig().get(HASH_MERGE);
+        if (hashMergeHint == null || !hashMergeHint.equalsIgnoreCase(TRUE_CONSTANT)) {
+            hashConnector = new HashPartitionExchangePOperator(vars, domain);
+            return hashConnector;
+        }
+        List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+        List<ILocalStructuralProperty> reqdLocals = requiredAtChild.getLocalProperties();
+        boolean propWasSet = false;
+        hashConnector = null;
+        if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+            AbstractLogicalOperator c = (AbstractLogicalOperator) parentOp.getInputs().get(childIndex).getValue();
+            Map<LogicalVariable, EquivalenceClass> ecs = ctx.getEquivalenceClassMap(c);
+            List<FunctionalDependency> fds = ctx.getFDList(c);
+            if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+                List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals, cldLocals);
+                hashConnector = new HashPartitionMergeExchangePOperator(orderColumns, vars, domain);
+                propWasSet = true;
+            }
+        }
+        if (!propWasSet) {
+            hashConnector = new HashPartitionExchangePOperator(vars, domain);
+        }
+        return hashConnector;
+    }
+
+    /**
+     * Creates a range-based exchange operator.
+     * @param parentOp the operator requiring range-based partitioner to have input tuples repartitioned using a range
+     * @param domain the target node domain of the range-based partitioner
+     * @param requiredPartitioning {@see OrderedPartitionedProperty}
+     * @param childIndex the index of the child at which the required partitioning is needed
+     * @param ctx optimization context
+     * @return a range-based exchange operator
+     * @throws AlgebricksException
+     */
+    private IPhysicalOperator createRangePartitionerConnector(AbstractLogicalOperator parentOp, INodeDomain domain,
+            IPartitioningProperty requiredPartitioning, int childIndex, IOptimizationContext ctx)
+            throws AlgebricksException {
+        // options for range partitioning: 1. static range map, 2. dynamic range map computed at run time
+        List<OrderColumn> partitioningColumns = ((OrderedPartitionedProperty) requiredPartitioning).getOrderColumns();
+        if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+            // TODO(ali): static range map implementation should be fixed to require ORDERED_PARTITION and come here.
+            RangeMap rangeMap = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+            return new RangePartitionExchangePOperator(partitioningColumns, domain, rangeMap);
+        } else {
+            return createDynamicRangePartitionExchangePOperator(parentOp, ctx, domain, partitioningColumns, childIndex);
+        }
+    }
+
+    private IPhysicalOperator createDynamicRangePartitionExchangePOperator(AbstractLogicalOperator parentOp,
+            IOptimizationContext ctx, INodeDomain targetDomain, List<OrderColumn> partitioningColumns, int childIndex)
+            throws AlgebricksException {
+        SourceLocation sourceLoc = parentOp.getSourceLocation();
+        // #1. create the replicate operator and add it above the source op feeding parent operator
+        ReplicateOperator replicateOp = createReplicateOperator(parentOp.getInputs().get(childIndex), ctx, sourceLoc);
+
+        // these two exchange ops are needed so that the parents of replicate stay the same during later optimizations.
+        // This is because replicate operator has references to its parents. If any later optimizations add new parents,
+        // then replicate would still point to the old ones.
+        MutableObject<ILogicalOperator> replicateOpRef = new MutableObject<>(replicateOp);
+        ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(replicateOpRef, ctx);
+        ExchangeOperator exchToForward = createOneToOneExchangeOp(replicateOpRef, ctx);
+        MutableObject<ILogicalOperator> exchToLocalAggRef = new MutableObject<>(exchToLocalAgg);
+        MutableObject<ILogicalOperator> exchToForwardRef = new MutableObject<>(exchToForward);
+
+        // add the exchange--to-forward at output 0, the exchange-to-local-aggregate at output 1
+        replicateOp.getOutputs().add(exchToForwardRef);
+        replicateOp.getOutputs().add(exchToLocalAggRef);
+        // materialize the data to be able to re-read the data again after sampling is done
+        replicateOp.getOutputMaterializationFlags()[0] = true;
+
+        // #2. create the aggregate operators and their sampling functions
+        // $$samplingResultVar = local_samplingFun($$partitioning_column)
+        // $$rangeMapResultVar = global_rangeMapFun($$samplingResultVar)
+        List<LogicalVariable> samplingResultVar = new ArrayList<>(1);
+        List<LogicalVariable> rangeMapResultVar = new ArrayList<>(1);
+        List<Mutable<ILogicalExpression>> samplingFun = new ArrayList<>(1);
+        List<Mutable<ILogicalExpression>> rangeMapFun = new ArrayList<>(1);
+
+        createAggregateFunction(ctx, samplingResultVar, samplingFun, rangeMapResultVar, rangeMapFun,
+                targetDomain.cardinality(), partitioningColumns, sourceLoc);
+
+        AggregateOperator localAggOp =
+                createAggregate(samplingResultVar, false, samplingFun, exchToLocalAggRef, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> localAgg = new MutableObject<>(localAggOp);
+        AggregateOperator globalAggOp = createAggregate(rangeMapResultVar, true, rangeMapFun, localAgg, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> globalAgg = new MutableObject<>(globalAggOp);
+
+        // #3. create the forward operator
+        String rangeMapKey = UUID.randomUUID().toString();
+        LogicalVariable rangeMapVar = rangeMapResultVar.get(0);
+        ForwardOperator forward = createForward(rangeMapKey, rangeMapVar, exchToForwardRef, globalAgg, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> forwardRef = new MutableObject<>(forward);
+
+        // replace the old input of parentOp requiring the range partitioning with the new forward op
+        parentOp.getInputs().set(childIndex, forwardRef);
+        parentOp.recomputeSchema();
+        ctx.computeAndSetTypeEnvironmentForOperator(parentOp);
+
+        return new RangePartitionExchangePOperator(partitioningColumns, rangeMapKey, targetDomain);
+    }
+
+    private static ReplicateOperator createReplicateOperator(Mutable<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        ReplicateOperator replicateOperator = new ReplicateOperator(2);
+        replicateOperator.setPhysicalOperator(new ReplicatePOperator());
+        replicateOperator.setSourceLocation(sourceLocation);
+        replicateOperator.getInputs().add(inputOperator);
+        OperatorManipulationUtil.setOperatorMode(replicateOperator);
+        replicateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(replicateOperator);
+        return replicateOperator;
+    }
+
+    /**
+     * Creates the sampling expressions and embeds them in {@code localAggFunctions} & {@code globalAggFunctions}. Also,
+     * creates the variables which will hold the result of each one.
+     * {@code localResultVariables},{@code localAggFunctions},{@code globalResultVariables} & {@code globalAggFunctions}
+     * will be used when creating the corresponding aggregate operators.
+     * @param context used to get new variables which will be assigned the samples & the range map
+     * @param localResultVariables the variable to which the stats (e.g. samples) info is assigned
+     * @param localAggFunctions the local sampling expression is added to this list
+     * @param globalResultVariables the variable to which the range map is assigned
+     * @param globalAggFunctions the expression generating a range map is added to this list
+     * @param numPartitions passed to the expression generating a range map to know how many split points are needed
+     * @param partFields the fields based on which the partitioner partitions the tuples, also sampled fields
+     * @param sourceLocation source location
+     */
+    private void createAggregateFunction(IOptimizationContext context, List<LogicalVariable> localResultVariables,
+            List<Mutable<ILogicalExpression>> localAggFunctions, List<LogicalVariable> globalResultVariables,
+            List<Mutable<ILogicalExpression>> globalAggFunctions, int numPartitions, List<OrderColumn> partFields,
+            SourceLocation sourceLocation) {
+        // prepare the arguments of the local sampling function: sampled fields
+        List<Mutable<ILogicalExpression>> sampledFields = new ArrayList<>(partFields.size());
+        partFields.forEach(f -> {
+            AbstractLogicalExpression sampledField = new VariableReferenceExpression(f.getColumn());
+            sampledField.setSourceLocation(sourceLocation);
+            sampledFields.add(new MutableObject<>(sampledField));
+        });
+
+        // local info
+        IFunctionInfo samplingFun = context.getMetadataProvider().lookupFunction(localSamplingFun);
+        AbstractFunctionCallExpression samplingExp =
+                new AggregateFunctionCallExpression(samplingFun, false, sampledFields);
+        samplingExp.setSourceLocation(sourceLocation);
+        LogicalVariable samplingResultVar = context.newVar();
+        localResultVariables.add(samplingResultVar);
+        localAggFunctions.add(new MutableObject<>(samplingExp));
+        Object[] samplingParam = { context.getPhysicalOptimizationConfig().getSortSamples() };
+        samplingExp.setOpaqueParameters(samplingParam);
+
+        // prepare the argument of the global range map generator function: the result of the local function
+        List<Mutable<ILogicalExpression>> arg = new ArrayList<>(1);
+        AbstractLogicalExpression samplingResultVarExp = new VariableReferenceExpression(samplingResultVar);
+        samplingResultVarExp.setSourceLocation(sourceLocation);
+        arg.add(new MutableObject<>(samplingResultVarExp));
+
+        // global info
+        IFunctionInfo rangeMapFun = context.getMetadataProvider().lookupFunction(rangeMapFunction);
+        AbstractFunctionCallExpression rangeMapExp = new AggregateFunctionCallExpression(rangeMapFun, true, arg);
+        rangeMapExp.setSourceLocation(sourceLocation);
+        globalResultVariables.add(context.newVar());
+        globalAggFunctions.add(new MutableObject<>(rangeMapExp));
+
+        int i = 0;
+        boolean[] ascendingFlags = new boolean[partFields.size()];
+        for (OrderColumn column : partFields) {
+            ascendingFlags[i] = column.getOrder() == OrderOperator.IOrder.OrderKind.ASC;
+            i++;
+        }
+        rangeMapExp.setOpaqueParameters(new Object[] { numPartitions, ascendingFlags });
+    }
+
+    /**
+     * Creates an aggregate operator. $$resultVariables = expressions()
+     * @param resultVariables the variables which stores the result of the aggregation
+     * @param isGlobal whether the aggregate operator is a global or local one
+     * @param expressions the aggregation functions desired
+     * @param inputOperator the input op that is feeding the aggregate operator
+     * @param context optimization context
+     * @param sourceLocation source location
+     * @return an aggregate operator with the specified information
+     * @throws AlgebricksException when there is error setting the type environment of the newly created aggregate op
+     */
+    private static AggregateOperator createAggregate(List<LogicalVariable> resultVariables, boolean isGlobal,
+            List<Mutable<ILogicalExpression>> expressions, MutableObject<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        AggregateOperator aggregateOperator = new AggregateOperator(resultVariables, expressions);
+        aggregateOperator.setPhysicalOperator(new AggregatePOperator());
+        aggregateOperator.setSourceLocation(sourceLocation);
+        aggregateOperator.getInputs().add(inputOperator);
+        aggregateOperator.setGlobal(isGlobal);
+        if (!isGlobal) {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+        } else {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        }
+        aggregateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+        return aggregateOperator;
+    }
+
+    private static ExchangeOperator createOneToOneExchangeOp(MutableObject<ILogicalOperator> inputOperator,
+            IOptimizationContext context) throws AlgebricksException {
+        ExchangeOperator exchangeOperator = new ExchangeOperator();
+        exchangeOperator.setPhysicalOperator(new OneToOneExchangePOperator());
+        exchangeOperator.getInputs().add(inputOperator);
+        exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        exchangeOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+        return exchangeOperator;
+    }
+
+    private static ForwardOperator createForward(String rangeMapKey, LogicalVariable rangeMapVariable,
+            MutableObject<ILogicalOperator> exchangeOpFromReplicate, MutableObject<ILogicalOperator> globalAggInput,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        AbstractLogicalExpression rangeMapExpression = new VariableReferenceExpression(rangeMapVariable);
+        rangeMapExpression.setSourceLocation(sourceLocation);
+        ForwardOperator forwardOperator = new ForwardOperator(rangeMapKey, new MutableObject<>(rangeMapExpression));
+        forwardOperator.setSourceLocation(sourceLocation);
+        forwardOperator.setPhysicalOperator(new ForwardPOperator());
+        forwardOperator.getInputs().add(exchangeOpFromReplicate);
+        forwardOperator.getInputs().add(globalAggInput);
+        OperatorManipulationUtil.setOperatorMode(forwardOperator);
+        forwardOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(forwardOperator);
+        return forwardOperator;
+    }
+
     private boolean allAreOrderProps(List<ILocalStructuralProperty> cldLocals) {
         for (ILocalStructuralProperty lsp : cldLocals) {
             if (lsp.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 3335d71..fa11f73 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -52,6 +52,10 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
+/**
+ * Pre-conditions:
+ *      FixReplicateOperatorOutputsRule should be fired
+ */
 public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
 
     private final HashMap<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> childrenToParents =
@@ -62,6 +66,8 @@
     private final HashMap<Mutable<ILogicalOperator>, MutableInt> clusterMap = new HashMap<>();
     private final HashMap<Integer, BitSet> clusterWaitForMap = new HashMap<>();
     private int lastUsedClusterId = 0;
+    private final Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs = new HashMap<>();
+    private final List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs = new ArrayList<>();
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -268,11 +274,71 @@
                     context.computeAndSetTypeEnvironmentForOperator(parentOp);
                 }
             }
+            cleanupPlan();
             rewritten = true;
         }
         return rewritten;
     }
 
+    /**
+     * Cleans up the plan after combining similar branches into one branch making sure parents & children point to
+     * each other correctly.
+     */
+    private void cleanupPlan() {
+        for (Mutable<ILogicalOperator> root : roots) {
+            replicateToOutputs.clear();
+            newOutputs.clear();
+            findReplicateOp(root, replicateToOutputs);
+            cleanup(replicateToOutputs, newOutputs);
+        }
+    }
+
+    /**
+     * Updates the outputs references of a replicate operator to points to the valid parents.
+     * @param replicateToOutputs where the replicate operators are stored with its valid parents.
+     * @param newOutputs the valid parents of replicate operator.
+     */
+    private void cleanup(Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs,
+            List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs) {
+        replicateToOutputs.forEach((repRef, allOutputs) -> {
+            newOutputs.clear();
+            // get the indexes that are set in the BitSet
+            allOutputs.stream().forEach(outIndex -> {
+                newOutputs.add(new Pair<>(((AbstractReplicateOperator) repRef.getValue()).getOutputs().get(outIndex),
+                        ((AbstractReplicateOperator) repRef.getValue()).getOutputMaterializationFlags()[outIndex]));
+            });
+            ((AbstractReplicateOperator) repRef.getValue()).setOutputs(newOutputs);
+        });
+    }
+
+    /**
+     * Collects all replicate operator starting from {@param parent} and all its descendants and keeps track of the
+     * valid parents of a replicate operator. The indexes of valid parents will be set in the BitSet.
+     * @param parent the current operator in consideration for which we want to find replicate op children.
+     * @param replicateToOutputs where the replicate operators will be stored with all its parents (valid & invalid).
+     */
+    private void findReplicateOp(Mutable<ILogicalOperator> parent,
+            Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs) {
+        List<Mutable<ILogicalOperator>> children = parent.getValue().getInputs();
+        for (Mutable<ILogicalOperator> childRef : children) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+            if (child.getOperatorTag() == LogicalOperatorTag.REPLICATE
+                    || child.getOperatorTag() == LogicalOperatorTag.SPLIT) {
+                AbstractReplicateOperator replicateChild = (AbstractReplicateOperator) child;
+                int parentIndex = replicateChild.getOutputs().indexOf(parent);
+                if (parentIndex >= 0) {
+                    BitSet replicateValidOutputs = replicateToOutputs.get(childRef);
+                    if (replicateValidOutputs == null) {
+                        replicateValidOutputs = new BitSet();
+                        replicateToOutputs.put(childRef, replicateValidOutputs);
+                    }
+                    replicateValidOutputs.set(parentIndex);
+                }
+            }
+            findReplicateOp(childRef, replicateToOutputs);
+        }
+    }
+
     private void genCandidates(IOptimizationContext context) throws AlgebricksException {
         List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
                 new ArrayList<List<Mutable<ILogicalOperator>>>();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index 6967271..5d6237a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -67,15 +67,18 @@
  */
 public class InlineVariablesRule implements IAlgebraicRewriteRule {
 
-    // Map of variables that could be replaced by their producing expression.
-    // Populated during the top-down sweep of the plan.
-    protected Map<LogicalVariable, ILogicalExpression> varAssignRhs = new HashMap<>();
-    // Visitor for replacing variable reference expressions with their originating expression.
+    // map of variables that could be replaced by their producing expression.
+    // populated during the top-down sweep of the plan.
+    private Map<LogicalVariable, ILogicalExpression> varAssignRhs = new HashMap<>();
+    // visitor for replacing variable reference expressions with their originating expression.
     protected InlineVariablesVisitor inlineVisitor = new InlineVariablesVisitor(varAssignRhs);
-    // Set of FunctionIdentifiers that we should not inline.
+    // set of FunctionIdentifiers that we should not inline.
     protected Set<FunctionIdentifier> doNotInlineFuncs = new HashSet<>();
-    // Indicates whether the rule has been run
-    protected boolean hasRun = false;
+    // indicates whether the rule has been run
+    private boolean hasRun = false;
+    // set to prevent re-visiting a subtree from the other sides. Operators with multiple outputs are the ones that
+    // could be re-visited twice or more (e.g. replicate and split operators)
+    private final Map<ILogicalOperator, Boolean> subTreesDone = new HashMap<>();
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -103,6 +106,7 @@
     protected void prepare(IOptimizationContext context) {
         varAssignRhs.clear();
         inlineVisitor.setContext(context);
+        subTreesDone.clear();
     }
 
     protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
@@ -118,10 +122,14 @@
         return false;
     }
 
-    protected boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    private boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
 
+        // check if you have already visited the subtree rooted at this operator
+        if (subTreesDone.containsKey(op)) {
+            return subTreesDone.get(op);
+        }
         // Update mapping from variables to expressions during top-down traversal.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
@@ -183,6 +191,10 @@
             // Re-enable rules that we may have already tried. They could be applicable now after inlining.
             context.removeFromAlreadyCompared(opRef.getValue());
         }
+        // mark the subtree rooted at op as visited so that you don't visit it again
+        if (op.getOperatorTag() == LogicalOperatorTag.REPLICATE || op.getOperatorTag() == LogicalOperatorTag.SPLIT) {
+            subTreesDone.put(op, modified);
+        }
 
         return modified;
     }
@@ -209,7 +221,7 @@
             this.context = context;
         }
 
-        public void setOperator(ILogicalOperator op) throws AlgebricksException {
+        public void setOperator(ILogicalOperator op) {
             this.op = op;
             liveVars.clear();
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 4869761..4273553 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -59,6 +59,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.DistributeResultPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ForwardPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexBulkloadPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexInsertDeleteUpsertPOperator;
@@ -394,6 +395,9 @@
                     op.setPhysicalOperator(new SinkPOperator());
                     break;
                 }
+                case FORWARD:
+                    op.setPhysicalOperator(new ForwardPOperator());
+                    break;
             }
         }
         if (op.hasNestedPlans()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index 35aa984..6b09894 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -34,10 +34,11 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -246,6 +247,11 @@
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return visit(op);
+    }
+
+    @Override
     public ILogicalOperator visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
         return visit(op);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
index 0dcc83a..2068ef3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
@@ -22,5 +22,23 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ITuplePartitionComputer {
+    /**
+     * For the tuple (located at tIndex in the frame), it determines which target partition (0,1,... nParts-1) the tuple
+     * should be sent/written to.
+     * @param accessor The accessor of the frame to access tuples
+     * @param tIndex The index of the tuple in consideration
+     * @param nParts The number of target partitions
+     * @return The chosen target partition number as dictated by the logic of the partition computer
+     * @throws HyracksDataException
+     */
     public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException;
+
+    /**
+     * Gives the data partitioner a chance to set up its environment before it starts partitioning tuples. This method
+     * should be called in the open() of {@link org.apache.hyracks.api.comm.IFrameWriter}. The default implementation
+     * is "do nothing".
+     * @throws HyracksDataException
+     */
+    public default void initialize() throws HyracksDataException {
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
index cde0057..81f9053 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
@@ -20,6 +20,8 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
 public interface ITuplePartitionComputerFactory extends Serializable {
-    public ITuplePartitionComputer createPartitioner();
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 09193d9..7d126ac 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -149,6 +149,9 @@
     public static final int UNDEFINED_INVERTED_LIST_MERGE_TYPE = 113;
     public static final int NODE_IS_NOT_ACTIVE = 114;
     public static final int LOCAL_NETWORK_ERROR = 115;
+    public static final int ONE_TUPLE_RANGEMAP_EXPECTED = 116;
+    public static final int NO_RANGEMAP_PRODUCED = 117;
+    public static final int RANGEMAP_NOT_FOUND = 118;
 
     // Compilation error codes.
     public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index c704d7e..50e92b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -132,6 +132,9 @@
 113 = Undefined inverted-list merge type: %1$s
 114 = Node (%1$s) is not active
 115 = Local network error
+116 = One tuple rangemap is expected
+117 = No range map produced for parallel sort
+118 = Range map was not found for parallel sort
 
 10000 = The given rule collection %1$s is not an instance of the List class.
 10001 = Cannot compose partition constraint %1$s with %2$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java
similarity index 94%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
rename to hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java
index b92aa6c..d858a7f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
+package org.apache.hyracks.data.std.util;
 
 import java.io.DataInputStream;
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java
similarity index 95%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java
rename to hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java
index 62e354b..2785751 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.library.java.base;
+package org.apache.hyracks.data.std.util;
 
 import java.io.ByteArrayInputStream;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
index dc66d19..ab5ab01 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.dataflow.common.data.partition;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -36,7 +37,7 @@
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
         for (int i = 0; i < hashFunctionFactories.length; ++i) {
             hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java
new file mode 100644
index 0000000..e55841a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.common.data.partition;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+
+public class OnePartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
+        return new ITuplePartitionComputer() {
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) {
+                return 0;
+            }
+        };
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
index e034af0..63d01fc 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
@@ -21,6 +21,7 @@
 import java.util.Random;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,7 +31,7 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         return new ITuplePartitionComputer() {
 
             private final Random random = new Random();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
index 9cb11fa..1821d78 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.dataflow.common.data.partition;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -35,9 +36,9 @@
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         return new ITuplePartitionComputer() {
-            private ITuplePartitionComputer delegate = delegateFactory.createPartitioner();
+            private ITuplePartitionComputer delegate = delegateFactory.createPartitioner(hyracksTaskContext);
 
             @Override
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java
new file mode 100644
index 0000000..bc642a9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.common.data.partition.range;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
+
+public class DynamicFieldRangePartitionComputerFactory extends FieldRangePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final String rangeMapKeyInContext;
+    private final SourceLocation sourceLocation;
+
+    public DynamicFieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
+            String rangeMapKeyInContext, SourceLocation sourceLocation) {
+        super(rangeFields, comparatorFactories);
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+        this.sourceLocation = sourceLocation;
+    }
+
+    @Override
+    protected RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) throws HyracksDataException {
+        RangeMap rangeMap = TaskUtil.get(rangeMapKeyInContext, hyracksTaskContext);
+        if (rangeMap == null) {
+            throw HyracksDataException.create(ErrorCode.RANGEMAP_NOT_FOUND, sourceLocation);
+        }
+        return rangeMap;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
index d58a248..55d4420 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
@@ -19,36 +19,41 @@
 package org.apache.hyracks.dataflow.common.data.partition.range;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
+public abstract class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
     private static final long serialVersionUID = 1L;
     private final int[] rangeFields;
-    private IRangeMap rangeMap;
     private IBinaryComparatorFactory[] comparatorFactories;
 
-    public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
-            IRangeMap rangeMap) {
+    protected FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories) {
         this.rangeFields = rangeFields;
         this.comparatorFactories = comparatorFactories;
-        this.rangeMap = rangeMap;
     }
 
+    protected abstract RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) throws HyracksDataException;
+
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+
         return new ITuplePartitionComputer() {
+            private RangeMap rangeMap;
+
             @Override
-            /**
-             * Determine the range partition.
-             */
+            public void initialize() throws HyracksDataException {
+                rangeMap = getRangeMap(hyracksTaskContext);
+            }
+
+            @Override
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
                 if (nParts == 1) {
                     return 0;
@@ -62,13 +67,10 @@
                 return (int) Math.floor(slotIndex / rangesPerPart);
             }
 
-            /*
-             * Determine the range partition.
-             */
-            public int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+            private int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
                 int slotIndex = 0;
-                for (int i = 0; i < rangeMap.getSplitCount(); ++i) {
-                    int c = compareSlotAndFields(accessor, tIndex, i);
+                for (int slotNumber = 0; slotNumber < rangeMap.getSplitCount(); ++slotNumber) {
+                    int c = compareSlotAndFields(accessor, tIndex, slotNumber);
                     if (c < 0) {
                         return slotIndex;
                     }
@@ -77,18 +79,18 @@
                 return slotIndex;
             }
 
-            public int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int fieldIndex)
+            private int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int slotNumber)
                     throws HyracksDataException {
                 int c = 0;
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int slotLength = accessor.getFieldSlotsLength();
-                for (int f = 0; f < comparators.length; ++f) {
-                    int fIdx = rangeFields[f];
+                for (int fieldNum = 0; fieldNum < comparators.length; ++fieldNum) {
+                    int fIdx = rangeFields[fieldNum];
                     int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
                     int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
-                    c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
-                            fEnd - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f),
-                            rangeMap.getLength(fieldIndex, f));
+                    c = comparators[fieldNum].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+                            fEnd - fStart, rangeMap.getByteArray(), rangeMap.getStartOffset(fieldNum, slotNumber),
+                            rangeMap.getLength(fieldNum, slotNumber));
                     if (c != 0) {
                         return c;
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
index 98acbc0..714e3c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
@@ -19,80 +19,110 @@
 package org.apache.hyracks.dataflow.common.data.partition.range;
 
 import java.io.Serializable;
-
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
+import java.util.Arrays;
+import java.util.Objects;
 
 /**
- * The range map stores the field split values in an byte array.
- * The first split value for each field followed by the second split value for each field, etc.
+ * <pre>
+ * The range map stores the fields split values in a byte array.
+ * The first split value for each field followed by the second split value for each field, etc. For example:
+ *                  split_point_idx0    split_point_idx1    split_point_idx2    split_point_idx3    split_point_idx4
+ * in the byte[]:   f0,f1,f2            f0,f1,f2            f0,f1,f2            f0,f1,f2            f0,f1,f2
+ * numFields would be = 3
+ * we have 5 split points, which gives us 6 partitions:
+ *      p1  |       p2      |       p3      |       p4      |       p5      |       p6
+ *          sp0             sp1             sp2             sp3             sp4
+ * endOffsets.length would be = 15
+ * </pre>
  */
-public class RangeMap implements IRangeMap, Serializable {
-    private final int fields;
+public class RangeMap implements Serializable {
+    private final int numFields;
     private final byte[] bytes;
-    private final int[] offsets;
+    private final int[] endOffsets;
 
-    public RangeMap(int fields, byte[] bytes, int[] offsets) {
-        this.fields = fields;
+    public RangeMap(int numFields, byte[] bytes, int[] endOffsets) {
+        this.numFields = numFields;
         this.bytes = bytes;
-        this.offsets = offsets;
+        this.endOffsets = endOffsets;
     }
 
-    @Override
-    public IPointable getFieldSplit(int columnIndex, int splitIndex) {
-        IPointable p = VoidPointable.FACTORY.createPointable();
-        int index = getFieldIndex(columnIndex, splitIndex);
-        p.set(bytes, getFieldStart(index), getFieldLength(index));
-        return p;
-    }
-
-    @Override
     public int getSplitCount() {
-        return offsets.length / fields;
+        return endOffsets.length / numFields;
     }
 
-    @Override
-    public byte[] getByteArray(int columnIndex, int splitIndex) {
+    public byte[] getByteArray() {
         return bytes;
     }
 
-    @Override
-    public int getTag(int columnIndex, int splitIndex) {
-        return getFieldTag(getFieldIndex(columnIndex, splitIndex));
+    public int getTag(int fieldIndex, int splitIndex) {
+        return getSplitValueTag(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    @Override
-    public int getStartOffset(int columnIndex, int splitIndex) {
-        return getFieldStart(getFieldIndex(columnIndex, splitIndex));
+    public int getStartOffset(int fieldIndex, int splitIndex) {
+        return getSplitValueStart(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    @Override
-    public int getLength(int columnIndex, int splitIndex) {
-        return getFieldLength(getFieldIndex(columnIndex, splitIndex));
+    public int getLength(int fieldIndex, int splitIndex) {
+        return getSplitValueLength(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    private int getFieldIndex(int columnIndex, int splitIndex) {
-        return splitIndex * fields + columnIndex;
+    /** Translates fieldIndex & splitIndex into an index which is used to find information about that split value.
+     * The combination of a fieldIndex & splitIndex uniquely identifies a split value of interest.
+     * @param fieldIndex the field index within the splitIndex of interest (0 <= fieldIndex < numFields)
+     * @param splitIndex starts with 0,1,2,.. etc
+     * @return the index of the desired split value that could be used with {@code bytes} & {@code endOffsets}.
+     */
+    private int getSplitValueIndex(int fieldIndex, int splitIndex) {
+        return splitIndex * numFields + fieldIndex;
     }
 
-    private int getFieldTag(int index) {
-        return bytes[getFieldStart(index)];
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the type tag of a specific field in a specific split point
+     */
+    private int getSplitValueTag(int splitValueIndex) {
+        return bytes[getSplitValueStart(splitValueIndex)];
     }
 
-    private int getFieldStart(int index) {
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the location of a split value in the byte array {@code bytes}
+     */
+    private int getSplitValueStart(int splitValueIndex) {
         int start = 0;
-        if (index != 0) {
-            start = offsets[index - 1];
+        if (splitValueIndex != 0) {
+            start = endOffsets[splitValueIndex - 1];
         }
         return start;
     }
 
-    private int getFieldLength(int index) {
-        int length = offsets[index];
-        if (index != 0) {
-            length -= offsets[index - 1];
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the length of a split value
+     */
+    private int getSplitValueLength(int splitValueIndex) {
+        int length = endOffsets[splitValueIndex];
+        if (splitValueIndex != 0) {
+            length -= endOffsets[splitValueIndex - 1];
         }
         return length;
     }
 
+    @Override
+    public int hashCode() {
+        return numFields + Arrays.hashCode(bytes) + Arrays.hashCode(endOffsets);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+        if (this == object) {
+            return true;
+        }
+        if (!(object instanceof RangeMap)) {
+            return false;
+        }
+        RangeMap other = (RangeMap) object;
+        return numFields == other.numFields && Arrays.equals(endOffsets, other.endOffsets)
+                && Arrays.equals(bytes, other.bytes);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java
new file mode 100644
index 0000000..b17c550
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.common.data.partition.range;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class StaticFieldRangePartitionComputerFactory extends FieldRangePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private RangeMap rangeMap;
+
+    public StaticFieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
+            RangeMap rangeMap) {
+        super(rangeFields, comparatorFactories);
+        this.rangeMap = rangeMap;
+    }
+
+    @Override
+    protected RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) {
+        return rangeMap;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
index 7d97507..8b57b15 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
@@ -20,7 +20,14 @@
 
 import java.util.BitSet;
 
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -47,4 +54,15 @@
     public boolean allProducersToAllConsumers() {
         return true;
     }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        NonDeterministicChannelReader channelReader =
+                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
index ab553f6..4c728ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
@@ -180,6 +180,7 @@
 
                 @Override
                 public void fail() throws HyracksDataException {
+                    // TODO: shouldn't we fail the MaterializerTaskState state?
                     HyracksDataException hde = null;
                     for (int i = 0; i < numberOfNonMaterializedOutputs; i++) {
                         if (isOpen[i]) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java
new file mode 100644
index 0000000..c437619
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.collectors;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hyracks.api.channels.IInputChannel;
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.partitions.PartitionId;
+
+public class DeterministicPartitionBatchManager implements IPartitionBatchManager {
+    private final IFrameReader[] partitions;
+    private List<IFrameReader> partitionsList;
+
+    public DeterministicPartitionBatchManager(int nSenders) {
+        this.partitions = new IFrameReader[nSenders];
+    }
+
+    @Override
+    public synchronized void addPartition(PartitionId partitionId, IInputChannel channel) {
+        InputChannelFrameReader channelReader = new InputChannelFrameReader(channel);
+        channel.registerMonitor(channelReader);
+        partitions[partitionId.getSenderIndex()] = channelReader;
+        if (allPartitionsAdded()) {
+            partitionsList = new ArrayList<>(Arrays.asList(partitions));
+            notifyAll();
+        }
+    }
+
+    @Override
+    public synchronized void getNextBatch(List<IFrameReader> batch, int requestedSize) throws HyracksDataException {
+        while (!allPartitionsAdded()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
+            }
+        }
+        if (partitionsList.isEmpty()) {
+            return;
+        }
+        if (requestedSize >= partitionsList.size()) {
+            batch.addAll(partitionsList);
+            partitionsList.clear();
+        } else {
+            List<IFrameReader> subBatch = partitionsList.subList(0, requestedSize);
+            batch.addAll(subBatch);
+            subBatch.clear();
+        }
+    }
+
+    private synchronized boolean allPartitionsAdded() {
+        for (int i = 0; i < partitions.length; i++) {
+            if (partitions[i] == null) {
+                return false;
+            }
+        }
+        return true;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java
new file mode 100644
index 0000000..2646c94
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.collectors;
+
+import java.util.LinkedList;
+
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SequentialMergeFrameReader implements IFrameReader {
+    private final int numSenders;
+    private final IPartitionBatchManager partitionBatchManager;
+    private final LinkedList<IFrameReader> senders;
+    private boolean isOpen;
+
+    public SequentialMergeFrameReader(int numSenders, IPartitionBatchManager partitionBatchManager) {
+        this.numSenders = numSenders;
+        this.partitionBatchManager = partitionBatchManager;
+        this.senders = new LinkedList<>();
+        this.isOpen = false;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (!isOpen) {
+            isOpen = true;
+            // get all the senders and open them one by one
+            partitionBatchManager.getNextBatch(senders, numSenders);
+            for (IFrameReader sender : senders) {
+                sender.open();
+            }
+        }
+    }
+
+    @Override
+    public boolean nextFrame(IFrame outFrame) throws HyracksDataException {
+        IFrameReader currentSender;
+        while (!senders.isEmpty()) {
+            // process the sender at the beginning of the sequence
+            currentSender = senders.getFirst();
+            outFrame.reset();
+            if (currentSender.nextFrame(outFrame)) {
+                return true;
+            } else {
+                // done with the current sender, close it, remove it from the Q and process the next one in sequence
+                currentSender.close();
+                senders.removeFirst();
+            }
+        }
+        // done with all senders
+        return false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (IFrameReader sender : senders) {
+            sender.close();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
index 920fdb8..0b6e40e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
@@ -61,7 +61,7 @@
     public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
-        return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(),
+        return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(ctx),
                 nConsumerPartitions, localityMap, index);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
index 092b5f1..32618ee 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
@@ -19,19 +19,14 @@
 package org.apache.hyracks.dataflow.std.connectors;
 
 import java.nio.ByteBuffer;
-import java.util.BitSet;
 
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.IPartitionCollector;
 import org.apache.hyracks.api.comm.IPartitionWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
-import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public class MToNBroadcastConnectorDescriptor extends AbstractMToNConnectorDescriptor {
 
@@ -123,15 +118,4 @@
             }
         };
     }
-
-    @Override
-    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
-            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
-        BitSet expectedPartitions = new BitSet(nProducerPartitions);
-        expectedPartitions.set(0, nProducerPartitions);
-        NonDeterministicChannelReader channelReader =
-                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
-        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
-        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
index 02fbedb..c11c08c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
@@ -18,10 +18,7 @@
  */
 package org.apache.hyracks.dataflow.std.connectors;
 
-import java.util.BitSet;
-
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.IPartitionCollector;
 import org.apache.hyracks.api.comm.IPartitionWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
@@ -29,9 +26,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
-import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -46,18 +40,7 @@
     public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
-        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner());
-    }
-
-    @Override
-    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
-            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
-        BitSet expectedPartitions = new BitSet(nProducerPartitions);
-        expectedPartitions.set(0, nProducerPartitions);
-        NonDeterministicChannelReader channelReader =
-                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
-        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
-        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
     }
 
     public ITuplePartitionComputerFactory getTuplePartitionComputerFactory() {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 026ca5e..e0ec5d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -70,7 +70,7 @@
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final PartitionDataWriter hashWriter =
-                new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner());
+                new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
         return hashWriter;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
index f6996f1..af3ce06 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
@@ -45,6 +45,6 @@
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new PartitionWithMessageDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
-                tpcf.createPartitioner());
+                tpcf.createPartitioner(ctx));
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java
new file mode 100644
index 0000000..3decb69
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.OnePartitionComputerFactory;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.collectors.DeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SequentialMergeFrameReader;
+
+public class MToOneSequentialMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final ITuplePartitionComputerFactory tpcf;
+
+    public MToOneSequentialMergingConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        super(spec);
+        tpcf = new OnePartitionComputerFactory();
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        // TODO(ali): create a single partition data writer instead
+        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IPartitionBatchManager pbm = new DeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sequentialMergeReader = new SequentialMergeFrameReader(nProducerPartitions, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sequentialMergeReader, pbm);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 5e33275..d06d5d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -113,6 +113,7 @@
 
     @Override
     public void open() throws HyracksDataException {
+        tpc.initialize();
         for (int i = 0; i < pWriters.length; ++i) {
             isOpen[i] = true;
             pWriters[i].open();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index dc250e6..034b054 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -190,7 +190,7 @@
                         ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
                 private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1);
                 private final ITuplePartitionComputer hpcBuild =
-                        new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                        new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                 private final FrameTupleAppender appender = new FrameTupleAppender();
                 private final FrameTupleAppender ftappender = new FrameTupleAppender();
                 private IFrame[] bufferForPartitions;
@@ -303,9 +303,9 @@
                     }
 
                     ITuplePartitionComputer hpc0 =
-                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner(ctx);
                     ITuplePartitionComputer hpc1 =
-                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                     int tableSize = (int) (state.memoryForHashtable * recordsPerFrame * factor);
                     ISerializableTable table = new SimpleSerializableHashTable(tableSize, ctx);
                     state.joiner =
@@ -385,7 +385,7 @@
                         new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories);
                 private final ITuplePartitionComputerFactory hpcf1 =
                         new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories);
-                private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner();
+                private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner(ctx);
 
                 private final FrameTupleAppender appender = new FrameTupleAppender();
                 private final FrameTupleAppender ftap = new FrameTupleAppender();
@@ -476,9 +476,9 @@
                             state.joiner.releaseMemory();
                         }
                         ITuplePartitionComputer hpcRep0 =
-                                new RepartitionComputerFactory(state.nPartitions, hpcf0).createPartitioner();
+                                new RepartitionComputerFactory(state.nPartitions, hpcf0).createPartitioner(ctx);
                         ITuplePartitionComputer hpcRep1 =
-                                new RepartitionComputerFactory(state.nPartitions, hpcf1).createPartitioner();
+                                new RepartitionComputerFactory(state.nPartitions, hpcf1).createPartitioner(ctx);
                         if (state.memoryForHashtable != memsize - 2) {
                             for (int i = 0; i < state.nPartitions; i++) {
                                 ByteBuffer buf = bufferForPartitions[i].getBuffer();
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 3873bae..a5c17f2 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -182,9 +182,9 @@
                 @Override
                 public void open() throws HyracksDataException {
                     ITuplePartitionComputer hpc0 =
-                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner(ctx);
                     ITuplePartitionComputer hpc1 =
-                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                     state = new HashBuildTaskState(ctx.getJobletContext().getJobId(),
                             new TaskId(getActivityId(), partition));
                     ISerializableTable table = new SerializableHashTable(tableSize, ctx, bufferManager);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java
new file mode 100644
index 0000000..24c5cae
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.misc;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class ForwardOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private static final int FORWARD_DATA_ACTIVITY_ID = 0;
+    private static final int RANGEMAP_READER_ACTIVITY_ID = 1;
+    private final String rangeMapKeyInContext;
+
+    /**
+     * @param spec used to create the operator id.
+     * @param rangeMapKeyInContext the unique key to store the range map in the shared map & transfer it to partitioner.
+     * @param outputRecordDescriptor the output schema of this operator.
+     */
+    public ForwardOperatorDescriptor(IOperatorDescriptorRegistry spec, String rangeMapKeyInContext,
+            RecordDescriptor outputRecordDescriptor) {
+        super(spec, 2, 1);
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+        outRecDescs[0] = outputRecordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ForwardDataActivity forwardDataActivity =
+                new ForwardDataActivity(new ActivityId(odId, FORWARD_DATA_ACTIVITY_ID));
+        RangeMapReaderActivity rangeMapReaderActivity =
+                new RangeMapReaderActivity(new ActivityId(odId, RANGEMAP_READER_ACTIVITY_ID));
+
+        // range map reader activity, its input is coming through the operator's in-port = 1 & activity's in-port = 0
+        builder.addActivity(this, rangeMapReaderActivity);
+        builder.addSourceEdge(1, rangeMapReaderActivity, 0);
+
+        // forward data activity, its input is coming through the operator's in-port = 0 & activity's in-port = 0
+        builder.addActivity(this, forwardDataActivity);
+        builder.addSourceEdge(0, forwardDataActivity, 0);
+
+        // forward data activity will wait for the range map reader activity
+        builder.addBlockingEdge(rangeMapReaderActivity, forwardDataActivity);
+
+        // data leaves from the operator's out-port = 0 & forward data activity's out-port = 0
+        builder.addTargetEdge(0, forwardDataActivity, 0);
+    }
+
+    /**
+     * Internal class that is used to transfer the {@link RangeMap} object between activities in different ctx but in
+     * the same NC, from {@link RangeMapReaderActivity} to {@link ForwardDataActivity}. These activities will share
+     * the {@link org.apache.hyracks.api.job.IOperatorEnvironment} of the {@link org.apache.hyracks.control.nc.Joblet}
+     * where the range map will be stored.
+     */
+    private class RangeMapState extends AbstractStateObject {
+        RangeMap rangeMap;
+
+        private RangeMapState(JobId jobId, TaskId stateObjectKey) {
+            super(jobId, stateObjectKey);
+        }
+    }
+
+    /**
+     * Range map reader activity. {@see {@link RangeMapReaderActivityNodePushable}}
+     */
+    private class RangeMapReaderActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private RangeMapReaderActivity(ActivityId activityId) {
+            super(activityId);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            RecordDescriptor inputRecordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new RangeMapReaderActivityNodePushable(ctx, inputRecordDescriptor, getActivityId(), partition);
+        }
+    }
+
+    /**
+     * Forward data activity. {@see {@link ForwardDataActivityNodePushable}}
+     */
+    private class ForwardDataActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private ForwardDataActivity(ActivityId activityId) {
+            super(activityId);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            return new ForwardDataActivityNodePushable(ctx, partition);
+        }
+    }
+
+    private class RangeMapReaderActivityNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+        private final FrameTupleAccessor frameTupleAccessor;
+        private final FrameTupleReference frameTupleReference;
+        private final IHyracksTaskContext ctx;
+        private final ActivityId activityId;
+        private final int partition;
+        private int numFields;
+        private byte[] splitValues;
+        private int[] splitValuesEndOffsets;
+
+        private RangeMapReaderActivityNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecordDescriptor,
+                ActivityId activityId, int partition) {
+            this.ctx = ctx;
+            this.frameTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+            this.frameTupleReference = new FrameTupleReference();
+            this.activityId = activityId;
+            this.partition = partition;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            // this activity does not have a consumer to open (it's a sink), and nothing to initialize
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            // "buffer" contains the serialized range map sent by a range map computer function.
+            // deserialize the range map
+            frameTupleAccessor.reset(buffer);
+            if (frameTupleAccessor.getTupleCount() != 1) {
+                throw HyracksDataException.create(ErrorCode.ONE_TUPLE_RANGEMAP_EXPECTED, sourceLoc);
+            }
+            frameTupleReference.reset(frameTupleAccessor, 0);
+            byte[] rangeMap = frameTupleReference.getFieldData(0);
+            int offset = frameTupleReference.getFieldStart(0);
+            int length = frameTupleReference.getFieldLength(0);
+
+            ByteArrayInputStream rangeMapIn = new ByteArrayInputStream(rangeMap, offset, length);
+            DataInputStream dataInputStream = new DataInputStream(rangeMapIn);
+            numFields = IntegerSerializerDeserializer.read(dataInputStream);
+            splitValues = ByteArraySerializerDeserializer.read(dataInputStream);
+            splitValuesEndOffsets = IntArraySerializerDeserializer.read(dataInputStream);
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            // it's a sink node pushable, nothing to fail
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // expecting a range map
+            if (numFields <= 0 || splitValues == null || splitValuesEndOffsets == null) {
+                throw HyracksDataException.create(ErrorCode.NO_RANGEMAP_PRODUCED, sourceLoc);
+            }
+            // store the range map in the state object of ctx so that next activity (forward) could retrieve it
+            TaskId rangeMapReaderTaskId = new TaskId(activityId, partition);
+            RangeMapState rangeMapState = new RangeMapState(ctx.getJobletContext().getJobId(), rangeMapReaderTaskId);
+            rangeMapState.rangeMap = new RangeMap(numFields, splitValues, splitValuesEndOffsets);
+            ctx.setStateObject(rangeMapState);
+        }
+    }
+
+    private class ForwardDataActivityNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+        private final IHyracksTaskContext ctx;
+        private final int partition;
+
+        /**
+         * @param ctx used to retrieve the range map stored by the range reader activity.
+         * @param partition used to create the same task id used by the range reader activity for storing the range.
+         */
+        private ForwardDataActivityNodePushable(IHyracksTaskContext ctx, int partition) {
+            this.ctx = ctx;
+            this.partition = partition;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            // retrieve the range map from the state object (previous activity should have already stored it)
+            // then deposit it into the ctx so that MToN-partition can pick it up
+            Object stateObjKey = new TaskId(new ActivityId(odId, RANGEMAP_READER_ACTIVITY_ID), partition);
+            RangeMapState rangeMapState = (RangeMapState) ctx.getStateObject(stateObjKey);
+            TaskUtil.put(rangeMapKeyInContext, rangeMapState.rangeMap, ctx);
+            writer.open();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            writer.nextFrame(buffer);
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            writer.fail();
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void flush() throws HyracksDataException {
+            writer.flush();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
index 4a77b3c..0bead97 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
@@ -111,7 +111,7 @@
 
                 int stop = runs.size();
                 currentGenerationRunAvailable.set(0, stop);
-
+                int numberOfPasses = 1;
                 while (true) {
 
                     int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
@@ -147,7 +147,7 @@
                         runs.add(reader);
 
                         if (currentGenerationRunAvailable.isEmpty()) {
-
+                            numberOfPasses++;
                             if (LOGGER.isDebugEnabled()) {
                                 LOGGER.debug("generated runs:" + stop);
                             }
@@ -157,7 +157,10 @@
                             stop = runs.size();
                         }
                     } else {
-                        LOGGER.debug("final runs: {}", stop);
+                        if (LOGGER.isDebugEnabled()) {
+                            LOGGER.debug("final runs: {}", stop);
+                            LOGGER.debug("number of passes: " + numberOfPasses);
+                        }
                         merge(finalWriter, partialRuns);
                         break;
                     }